diff --git a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs index b833e696c52..de849013dab 100644 --- a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs +++ b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs @@ -22,15 +22,15 @@ sealed class CustomTxnSet : CustomTransactionProcedure /// public const string CommandName = "CTXNSET"; - ArgSlice setA; - ArgSlice setB; - ArgSlice setC; - ArgSlice setD; + PinnedSpanByte setA; + PinnedSpanByte setB; + PinnedSpanByte setC; + PinnedSpanByte setD; - ArgSlice valueA; - ArgSlice valueB; - ArgSlice valueC; - ArgSlice valueD; + PinnedSpanByte valueA; + PinnedSpanByte valueB; + PinnedSpanByte valueC; + PinnedSpanByte valueD; /// /// CTXNSET key1 key2 key3 key4 value1 value2 value3 value4 diff --git a/libs/client/ClientSession/GarnetClientSession.cs b/libs/client/ClientSession/GarnetClientSession.cs index c822a17de21..5242130a1c3 100644 --- a/libs/client/ClientSession/GarnetClientSession.cs +++ b/libs/client/ClientSession/GarnetClientSession.cs @@ -559,12 +559,17 @@ private unsafe void Flush() Dispose(); throw; } - networkSender.GetResponseObject(); - offset = networkSender.GetResponseObjectHead(); - end = networkSender.GetResponseObjectTail(); + ResetOffset(); } } + private unsafe void ResetOffset() + { + networkSender.GetResponseObject(); + offset = networkSender.GetResponseObjectHead(); + end = networkSender.GetResponseObjectTail(); + } + /// public bool TryCreateMessageConsumer(Span bytesReceived, INetworkSender networkSender, out IMessageConsumer session) => throw new NotSupportedException(); diff --git a/libs/client/ClientSession/GarnetClientSessionIncremental.cs b/libs/client/ClientSession/GarnetClientSessionIncremental.cs index d12a4e12218..54be09a4452 100644 --- a/libs/client/ClientSession/GarnetClientSessionIncremental.cs +++ b/libs/client/ClientSession/GarnetClientSessionIncremental.cs @@ -23,7 +23,7 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo IncrementalSendType ist; bool isMainStore; byte* curr, head; - int keyValuePairCount; + int recordCount; TaskCompletionSource currTcsIterationTask = null; /// @@ -42,15 +42,37 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo public bool NeedsInitialization => curr == null; /// - /// Flush and initialize buffers/parameters used for migrate command + /// Return a of all remaining available space in the network buffer. + /// + public PinnedSpanByte GetAvailableNetworkBufferSpan() => PinnedSpanByte.FromPinnedPointer(curr, (int)(end - curr)); + + public void IncrementRecordDirect(int size) + { + ++recordCount; + curr += size; + } + + private void EnsureTcsIsEnqueued() + { + // See comments in SetClusterMigrateHeader() as to why this is decoupled from the header initialization. + if (recordCount > 0 && currTcsIterationTask == null) + { + currTcsIterationTask = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + tcsQueue.Enqueue(currTcsIterationTask); + } + } + + /// + /// Flush and initialize buffers/parameters used for Migrate and Replica commands /// /// public void InitializeIterationBuffer(TimeSpan iterationProgressFreq = default) { + EnsureTcsIsEnqueued(); Flush(); currTcsIterationTask = null; curr = head = null; - keyValuePairCount = 0; + recordCount = 0; this.iterationProgressFreq = default ? TimeSpan.FromSeconds(5) : iterationProgressFreq; } @@ -59,7 +81,14 @@ public void InitializeIterationBuffer(TimeSpan iterationProgressFreq = default) /// public Task SendAndResetIterationBuffer() { - if (keyValuePairCount == 0) return null; + Task task = null; + if (recordCount == 0) + { + // No records to Flush(), but we need to reset buffer offsets as we may have written a header due to the need to initialize the buffer + // before passing it to Tsavorite as the output SpanByteAndMemory.SpanByte for Read(). + ResetOffset(); + goto done; + } Debug.Assert(end - curr >= 2); *curr++ = (byte)'\r'; @@ -67,109 +96,49 @@ public Task SendAndResetIterationBuffer() // Payload format = [$length\r\n][number of keys (4 bytes)][raw key value pairs]\r\n var size = (int)(curr - 2 - head - (ExtraSpace - 4)); - TrackIterationProgress(keyValuePairCount, size); + TrackIterationProgress(recordCount, size); var success = RespWriteUtils.TryWritePaddedBulkStringLength(size, ExtraSpace - 4, ref head, end); Debug.Assert(success); // Number of key value pairs in payload - *(int*)head = keyValuePairCount; + *(int*)head = recordCount; // Reset offset and flush buffer offset = curr; + EnsureTcsIsEnqueued(); Flush(); Interlocked.Increment(ref numCommands); // Return outstanding task and reset current tcs - var task = currTcsIterationTask.Task; + task = currTcsIterationTask.Task; currTcsIterationTask = null; + recordCount = 0; + + done: curr = head = null; - keyValuePairCount = 0; return task; } /// - /// Try write key value pair for main store directly to the client buffer + /// Try to write the span for the entire record directly to the client buffer /// - /// - /// - /// - /// - public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out Task task) + public bool TryWriteRecordSpan(ReadOnlySpan recordSpan, out Task task) { - task = null; - // Try write key value pair directly to client buffer - if (!WriteSerializedSpanByte(ref key, ref value)) + // We include space for newline at the end, to be added before sending + var totalLen = recordSpan.TotalSize() + 2; + if (totalLen > (int)(end - curr)) { - // If failed to write because no space left send outstanding data and retrieve task - // Caller is responsible for retrying + // If there is no space left, send outstanding data and return the send-completion task. + // Caller is responsible for waiting for task completion and retrying. task = SendAndResetIterationBuffer(); return false; } - keyValuePairCount++; - return true; - - bool WriteSerializedSpanByte(ref SpanByte key, ref SpanByte value) - { - var totalLen = key.TotalSize + value.TotalSize + 2 + 2; - if (totalLen > (int)(end - curr)) - return false; - - key.CopyTo(curr); - curr += key.TotalSize; - value.CopyTo(curr); - curr += value.TotalSize; - return true; - } - } - - /// - /// Try write key value pair for object store directly to the client buffer - /// - /// - /// - /// - /// - /// - public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, out Task task) - { + recordSpan.SerializeTo(curr); + curr += recordSpan.TotalSize(); + ++recordCount; task = null; - // Try write key value pair directly to client buffer - if (!WriteSerializedKeyValueByteArray(key, value, expiration)) - { - // If failed to write because no space left send outstanding data and retrieve task - // Caller is responsible for retrying - task = SendAndResetIterationBuffer(); - return false; - } - - keyValuePairCount++; return true; - - bool WriteSerializedKeyValueByteArray(byte[] key, byte[] value, long expiration) - { - // We include space for newline at the end, to be added before sending - int totalLen = 4 + key.Length + 4 + value.Length + 8 + 2; - if (totalLen > (int)(end - curr)) - return false; - - *(int*)curr = key.Length; - curr += 4; - fixed (byte* keyPtr = key) - Buffer.MemoryCopy(keyPtr, curr, key.Length, key.Length); - curr += key.Length; - - *(int*)curr = value.Length; - curr += 4; - fixed (byte* valPtr = value) - Buffer.MemoryCopy(valPtr, curr, value.Length, value.Length); - curr += value.Length; - - *(long*)curr = expiration; - curr += 8; - - return true; - } } long lastLog = 0; diff --git a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs index 7662b533f83..56a29637289 100644 --- a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs @@ -172,8 +172,10 @@ public Task SetSlotRange(Memory state, string nodeid, List<(int, i /// public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMainStore) { - currTcsIterationTask = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - tcsQueue.Enqueue(currTcsIterationTask); + // For Migration we send the (curr - end) buffer as the SpanByteAndMemory.SpanByte output to Tsavorite. Thus we must + // initialize the header first, so we have curr properly positioned, but we cannot yet enqueue currTcsIterationTask. + // Therefore we defer this until the actual Flush(), when we know we have records to send. This is not a concern for + // Replication, because it uses an iterator and thus knows it has a record before it initializes the header. curr = offset; this.isMainStore = isMainStore; this.ist = IncrementalSendType.MIGRATE; @@ -257,15 +259,16 @@ public Task CompleteMigrate(string sourceNodeId, bool replace, bool isMa // Payload format = [$length\r\n][number of keys (4 bytes)][raw key value pairs]\r\n var size = (int)(curr - 2 - head - (ExtraSpace - 4)); - TrackIterationProgress(keyValuePairCount, size, completed: true); + TrackIterationProgress(recordCount, size, completed: true); var success = RespWriteUtils.TryWritePaddedBulkStringLength(size, ExtraSpace - 4, ref head, end); Debug.Assert(success); // Number of key value pairs in payload - *(int*)head = keyValuePairCount; + *(int*)head = recordCount; // Reset offset and flush buffer offset = curr; + EnsureTcsIsEnqueued(); Flush(); Interlocked.Increment(ref numCommands); @@ -273,7 +276,7 @@ public Task CompleteMigrate(string sourceNodeId, bool replace, bool isMa var task = currTcsIterationTask.Task; currTcsIterationTask = null; curr = head = null; - keyValuePairCount = 0; + recordCount = 0; return task; } } diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs index 408471734bf..2f9f060c415 100644 --- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs @@ -410,6 +410,8 @@ 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 + // we know we have a record at the time this is called, so we can initialize it directly. currTcsIterationTask = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); tcsQueue.Enqueue(currTcsIterationTask); curr = offset; diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs index a35e474a263..5041f656d9a 100644 --- a/libs/cluster/Server/ClusterManagerSlotState.cs +++ b/libs/cluster/Server/ClusterManagerSlotState.cs @@ -12,12 +12,12 @@ namespace Garnet.cluster { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; /// /// Cluster manager @@ -477,12 +477,12 @@ public void TryResetSlotState(HashSet slots) public static unsafe void DeleteKeysInSlotsFromMainStore(BasicGarnetApi BasicGarnetApi, HashSet slots) { using var iter = BasicGarnetApi.IterateMainStore(); - while (iter.GetNext(out _)) + while (iter.GetNext()) { - ref var key = ref iter.GetKey(); - var s = HashSlotUtils.HashSlot(ref key); + var key = iter.Key; + var s = HashSlotUtils.HashSlot(key); if (slots.Contains(s)) - _ = BasicGarnetApi.DELETE(ref key, StoreType.Main); + _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Main); } } @@ -494,13 +494,12 @@ public static unsafe void DeleteKeysInSlotsFromMainStore(BasicGarnetApi BasicGar public static unsafe void DeleteKeysInSlotsFromObjectStore(BasicGarnetApi BasicGarnetApi, HashSet slots) { using var iterObject = BasicGarnetApi.IterateObjectStore(); - while (iterObject.GetNext(out _)) + while (iterObject.GetNext()) { - ref var key = ref iterObject.GetKey(); - ref var value = ref iterObject.GetValue(); + var key = iterObject.Key; var s = HashSlotUtils.HashSlot(key); if (slots.Contains(s)) - _ = BasicGarnetApi.DELETE(key, StoreType.Object); + _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Object); } } } diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs index 906789b45ad..2535f2aa782 100644 --- a/libs/cluster/Server/ClusterProvider.cs +++ b/libs/cluster/Server/ClusterProvider.cs @@ -15,12 +15,12 @@ namespace Garnet.cluster { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; /// /// Cluster provider diff --git a/libs/cluster/Server/Migration/MigrateSessionKeyManager.cs b/libs/cluster/Server/Migration/MigrateSessionKeyManager.cs index 44ecc33b2c5..424be43381c 100644 --- a/libs/cluster/Server/Migration/MigrateSessionKeyManager.cs +++ b/libs/cluster/Server/Migration/MigrateSessionKeyManager.cs @@ -4,7 +4,7 @@ using System; using System.Runtime.CompilerServices; using Garnet.common; -using Garnet.server; +using Tsavorite.core; namespace Garnet.cluster { @@ -59,14 +59,14 @@ private void TryTransitionState(KeyMigrationStatus status) /// /// /// - public bool CanAccessKey(ref ArgSlice key, int slot, bool readOnly) + public bool CanAccessKey(ref PinnedSpanByte key, int slot, bool readOnly) { // Skip operation check since this session is not responsible for migrating the associated slot if (!_sslots.Contains(slot)) return true; // If key is not queued for migration then - if (!_keys.TryGetValue(ref key, out var state)) + if (!_keys.TryGetValue(key, out var state)) return true; // NOTE: @@ -86,8 +86,8 @@ public bool CanAccessKey(ref ArgSlice key, int slot, bool readOnly) /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool AddKey(ref ArgSlice key) - => _keys.TryAdd(ref key, KeyMigrationStatus.QUEUED); + public bool AddKey(PinnedSpanByte key) + => _keys.TryAdd(key, KeyMigrationStatus.QUEUED); /// /// Clear keys from dictionary diff --git a/libs/cluster/Server/Migration/MigrateSessionKeys.cs b/libs/cluster/Server/Migration/MigrateSessionKeys.cs index f19f731fb94..fcd0ec2eced 100644 --- a/libs/cluster/Server/Migration/MigrateSessionKeys.cs +++ b/libs/cluster/Server/Migration/MigrateSessionKeys.cs @@ -20,60 +20,33 @@ internal sealed unsafe partial class MigrateSession : IDisposable /// True on success, false otherwise private bool MigrateKeysFromMainStore() { - var bufferSize = 1 << 10; - SectorAlignedMemory buffer = new(bufferSize, 1); - var bufPtr = buffer.GetValidPointer(); - var bufPtrEnd = bufPtr + bufferSize; - var o = new SpanByteAndMemory(bufPtr, (int)(bufPtrEnd - bufPtr)); + var input = new RawStringInput(RespCommandAccessor.MIGRATE); + var output = new SpanByteAndMemory(); try { - // Transition keys to MIGRATING status + // NOTE: Any keys not found in main store are automatically set to QUEUED before this method is called + // Transition all QUEUED to MIGRATING state TryTransitionState(KeyMigrationStatus.MIGRATING); WaitForConfigPropagation(); - //////////////// - // Build Input// - //////////////// - var input = new RawStringInput(RespCommandAccessor.MIGRATE); + // Must initialize this here because we use the network buffer as output. + if (_gcs.NeedsInitialization) + _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true); foreach (var pair in _keys.GetKeys()) { - // Process only keys in MIGRATING status if (pair.Value != KeyMigrationStatus.MIGRATING) continue; - var key = pair.Key.SpanByte; - - // Read value for key - var status = localServerSession.BasicGarnetApi.Read_MainStore(ref key, ref input, ref o); - - // Check if found in main store - if (status == GarnetStatus.NOTFOUND) - { - // Transition key status back to QUEUED to unblock any writers - _keys.UpdateStatus(pair.Key, KeyMigrationStatus.QUEUED); - continue; - } - - // Get SpanByte from stack if any - ref var value = ref o.SpanByte; - if (!o.IsSpanByte) - { - // Reinterpret heap memory to SpanByte - value = ref SpanByte.ReinterpretWithoutLength(o.Memory.Memory.Span); - } - - // Write key to network buffer if it has not expired - if (!ClusterSession.Expired(ref value) && !WriteOrSendMainStoreKeyValuePair(ref key, ref value)) + // Read the value for the key. This will populate output with the entire serialized record. + output.SpanByte = _gcs.GetAvailableNetworkBufferSpan(); + var status = localServerSession.BasicGarnetApi.Read_MainStore(pair.Key, ref input, ref output); + if (!WriteRecord(pair.Key, ref output, status, isMainStore: true)) return false; - - // Reset SpanByte for next read if any but don't dispose heap buffer as we might re-use it - o.SpanByte = new SpanByte((int)(bufPtrEnd - bufPtr), (IntPtr)bufPtr); } - // Flush data in client buffer - if (!HandleMigrateTaskResponse(_gcs.SendAndResetIterationBuffer())) + if (!FlushFinalMigrationBuffer(isMainStore: true)) return false; DeleteKeys(); @@ -81,9 +54,7 @@ private bool MigrateKeysFromMainStore() finally { // If allocated memory in heap dispose it here. - if (o.Memory != default) - o.Memory.Dispose(); - buffer.Dispose(); + output.Dispose(); } return true; } @@ -95,51 +66,64 @@ private bool MigrateKeysFromMainStore() /// True on success, false otherwise private bool MigrateKeysFromObjectStore() { + var input = new ObjectInput(new RespInputHeader(GarnetObjectType.Migrate)); + var output = new GarnetObjectStoreOutput(); + try { - // NOTE: Any keys not found in main store are automatically set to QUEUED before this method is called - // Transition all QUEUED to MIGRATING state + // Transition keys to MIGRATING status. + // NOTE: Any keys not found in main store are automatically set to QUEUED. TryTransitionState(KeyMigrationStatus.MIGRATING); WaitForConfigPropagation(); - foreach (var mKey in _keys.GetKeys()) + // Must initialize this here because we use the network buffer as output. + if (_gcs.NeedsInitialization) + _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false); + + foreach (var pair in _keys.GetKeys()) { - // Process only keys in MIGRATING status - if (mKey.Value != KeyMigrationStatus.MIGRATING) - continue; - var key = mKey.Key.ToArray(); - - ObjectInput input = default; - GarnetObjectStoreOutput value = default; - var status = localServerSession.BasicGarnetApi.Read_ObjectStore(ref key, ref input, ref value); - if (status == GarnetStatus.NOTFOUND) - { - // Transition key status back to QUEUED to unblock any writers - _keys.UpdateStatus(mKey.Key, KeyMigrationStatus.QUEUED); + if (pair.Value != KeyMigrationStatus.MIGRATING) continue; - } - - if (!ClusterSession.Expired(ref value.GarnetObject)) - { - var objectData = GarnetObjectSerializer.Serialize(value.GarnetObject); - if (!WriteOrSendObjectStoreKeyValuePair(key, objectData, value.GarnetObject.Expiration)) - return false; - } + // Read the value for the key. This will populate output with the entire serialized record. + output.SpanByteAndMemory.SpanByte = _gcs.GetAvailableNetworkBufferSpan(); + var status = localServerSession.BasicGarnetApi.Read_ObjectStore(pair.Key, ref input, ref output); + if (!WriteRecord(pair.Key, ref output.SpanByteAndMemory, status, isMainStore: false)) + return false; } - // Flush data in client buffer - if (!HandleMigrateTaskResponse(_gcs.SendAndResetIterationBuffer())) + if (!FlushFinalMigrationBuffer(isMainStore: false)) return false; } finally { // Delete keys if COPY option is false or transition KEYS from MIGRATING to MIGRATED status DeleteKeys(); + + output.SpanByteAndMemory.Dispose(); } return true; } + bool WriteRecord(PinnedSpanByte key, ref SpanByteAndMemory output, GarnetStatus status, bool isMainStore) + { + if (status == GarnetStatus.NOTFOUND) + { + // Transition key status back to QUEUED to unblock any writers + _keys.UpdateStatus(key, KeyMigrationStatus.QUEUED); + return true; + } + + // If the SBAM is still SpanByte then there was enough room to write directly to the network buffer, so increment curr and + // there is nothing more to do for this key. Otherwise, we need to Flush() and copy to the network buffer. + if (output.IsSpanByte) + { + _gcs.IncrementRecordDirect(output.SpanByte.TotalSize); + return true; + } + return WriteOrSendRecordSpan(ref output, isMainStore); + } + /// /// Delete local copy of keys if _copyOption is set to false. /// @@ -162,8 +146,8 @@ private void DeleteKeys() if (mKey.Value != KeyMigrationStatus.DELETING) continue; - var key = mKey.Key.SpanByte; - _ = localServerSession.BasicGarnetApi.DELETE(ref key); + var key = mKey.Key; + _ = localServerSession.BasicGarnetApi.DELETE(key); // Set key as MIGRATED to allow allow all operations _keys.UpdateStatus(mKey.Key, KeyMigrationStatus.MIGRATED); diff --git a/libs/cluster/Server/Migration/MigrateSessionSend.cs b/libs/cluster/Server/Migration/MigrateSessionSend.cs index 9b9a1ad8020..01909218e31 100644 --- a/libs/cluster/Server/Migration/MigrateSessionSend.cs +++ b/libs/cluster/Server/Migration/MigrateSessionSend.cs @@ -11,51 +11,39 @@ namespace Garnet.cluster internal sealed unsafe partial class MigrateSession : IDisposable { /// - /// Write main store key-value pair directly to client buffer or flush buffer to make space and try again writing. + /// Write a serialized record directly to the client buffer; if there is not enough room, flush the buffer and retry writing. /// - /// - /// + /// Output buffer from Read(), containing the full serialized record /// True on success, else false - private bool WriteOrSendMainStoreKeyValuePair(ref SpanByte key, ref SpanByte value) + private bool WriteOrSendRecordSpan(ref SpanByteAndMemory output, bool isMainStore) { // Check if we need to initialize cluster migrate command arguments if (_gcs.NeedsInitialization) - _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true); + _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore); - // Try write serialized key value to client buffer - while (!_gcs.TryWriteKeyValueSpanByte(ref key, ref value, out var task)) + fixed (byte* ptr = output.MemorySpan) { - // Flush key value pairs in the buffer - if (!HandleMigrateTaskResponse(task)) - return false; + // Try to write serialized record to client buffer + while (!_gcs.TryWriteRecordSpan(new(ptr, output.Length), out var task)) + { + // Flush records in the buffer + if (!HandleMigrateTaskResponse(task)) + return false; - // re-initialize cluster migrate command parameters - _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true); + // Re-initialize cluster migrate command parameters for the next loop iteration + _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore); + } } return true; } /// - /// Write object store key-value pair directly to client buffer or flush buffer to make space and try again writing. + /// Flush the final partial buffer to the client. /// - /// - /// - /// - /// - private bool WriteOrSendObjectStoreKeyValuePair(byte[] key, byte[] value, long expiration) + /// True on success, else false + private bool FlushFinalMigrationBuffer(bool isMainStore) { - // Check if we need to initialize cluster migrate command arguments - if (_gcs.NeedsInitialization) - _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false); - - while (!_gcs.TryWriteKeyValueByteArray(key, value, expiration, out var task)) - { - // Flush key value pairs in the buffer - if (!HandleMigrateTaskResponse(task)) - return false; - _gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false); - } - return true; + return HandleMigrateTaskResponse(_gcs.SendAndResetIterationBuffer()); } /// diff --git a/libs/cluster/Server/Migration/MigrateSessionTaskStore.cs b/libs/cluster/Server/Migration/MigrateSessionTaskStore.cs index 953311c0270..300364606e1 100644 --- a/libs/cluster/Server/Migration/MigrateSessionTaskStore.cs +++ b/libs/cluster/Server/Migration/MigrateSessionTaskStore.cs @@ -5,8 +5,8 @@ using System.Collections.Generic; using System.Diagnostics; using Garnet.common; -using Garnet.server; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.cluster { @@ -211,7 +211,7 @@ public bool TryRemove(string targetNodeId) /// /// /// True if we can operate on the key, otherwise false (i.e. key is being migrated) - public bool CanAccessKey(ref ArgSlice key, int slot, bool readOnly) + public bool CanAccessKey(PinnedSpanByte key, int slot, bool readOnly) { try { diff --git a/libs/cluster/Server/Migration/MigratingKeysWorkingSet.cs b/libs/cluster/Server/Migration/MigratingKeysWorkingSet.cs index d8dd657e9e9..44e37fda4d3 100644 --- a/libs/cluster/Server/Migration/MigratingKeysWorkingSet.cs +++ b/libs/cluster/Server/Migration/MigratingKeysWorkingSet.cs @@ -4,21 +4,21 @@ using System.Collections.Generic; using System.Runtime.CompilerServices; using Garnet.common; -using Garnet.server; +using Tsavorite.core; namespace Garnet.cluster { internal class MigratingKeysWorkingSet { - readonly Dictionary WorkingSet; + readonly Dictionary WorkingSet; SingleWriterMultiReaderLock keyDictLock; public MigratingKeysWorkingSet() { - WorkingSet = new Dictionary(ArgSliceComparer.Instance); + WorkingSet = new Dictionary(PinnedSpanByteComparer.Instance); } - public IEnumerable> GetKeys() + public IEnumerable> GetKeys() { foreach (var pair in WorkingSet) yield return pair; @@ -36,7 +36,7 @@ public bool IsNullOrEmpty() /// /// /// - public bool TryAdd(ref ArgSlice key, KeyMigrationStatus status) + public bool TryAdd(PinnedSpanByte key, KeyMigrationStatus status) { try { @@ -55,7 +55,7 @@ public bool TryAdd(ref ArgSlice key, KeyMigrationStatus status) /// /// /// - public bool TryGetValue(ref ArgSlice key, out KeyMigrationStatus status) + public bool TryGetValue(PinnedSpanByte key, out KeyMigrationStatus status) { try { @@ -77,7 +77,7 @@ public bool TryGetValue(ref ArgSlice key, out KeyMigrationStatus status) /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UpdateStatus(ArgSlice key, KeyMigrationStatus status) + public void UpdateStatus(PinnedSpanByte key, KeyMigrationStatus status) => WorkingSet[key] = status; /// diff --git a/libs/cluster/Server/Migration/MigrationKeyIterationFunctions.cs b/libs/cluster/Server/Migration/MigrationKeyIterationFunctions.cs index 7587afcec8d..c8e1ae5923a 100644 --- a/libs/cluster/Server/Migration/MigrationKeyIterationFunctions.cs +++ b/libs/cluster/Server/Migration/MigrationKeyIterationFunctions.cs @@ -5,7 +5,6 @@ using System.Collections.Generic; using System.Runtime.CompilerServices; using Garnet.common; -using Garnet.server; using Tsavorite.core; namespace Garnet.cluster @@ -14,7 +13,7 @@ internal sealed unsafe partial class MigrateSession : IDisposable { internal sealed class MigrationKeyIterationFunctions { - internal sealed unsafe class MainStoreGetKeysInSlots : IScanIteratorFunctions + internal sealed unsafe class MainStoreGetKeysInSlots : IScanIteratorFunctions { MigrationScanIterator iterator; @@ -30,34 +29,28 @@ internal void Dispose() public void AdvanceIterator() => iterator.AdvanceIterator(); - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here // Do not send key if it is expired - if (ClusterSession.Expired(ref value)) + if (ClusterSession.Expired(ref srcLogRecord)) return true; - var s = HashSlotUtils.HashSlot(ref key); - // Transfer key if it belongs to slot that is currently being migrated - if (iterator.Contains(s)) - { - var keySpan = key.AsSpan(); - if (!iterator.Consume(ref keySpan)) - return false; - } + var key = srcLogRecord.Key; + var slot = HashSlotUtils.HashSlot(key); - return true; + // Transfer key if it belongs to slot that is currently being migrated + return !iterator.Contains(slot) || iterator.Consume(key); } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } } - internal struct ObjectStoreGetKeysInSlots : IScanIteratorFunctions + internal struct ObjectStoreGetKeysInSlots : IScanIteratorFunctions { MigrationScanIterator iterator; @@ -73,28 +66,22 @@ internal void Dispose() public void AdvanceIterator() => iterator.AdvanceIterator(); - public bool SingleReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here // Do not send key if it is expired - if (ClusterSession.Expired(ref value)) + if (ClusterSession.Expired(ref srcLogRecord)) return true; - var s = HashSlotUtils.HashSlot(key); - // Transfer key if it belongs to slot that is currently being migrated - if (iterator.Contains(s)) - { - var keySpan = key.AsSpan(); - if (!iterator.Consume(ref keySpan)) - return false; - } + var key = srcLogRecord.Key; + var slot = HashSlotUtils.HashSlot(key); - return true; + // Transfer key if it belongs to slot that is currently being migrated + return !iterator.Contains(slot) || iterator.Consume(key); } - public bool ConcurrentReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } @@ -154,7 +141,7 @@ public void AdvanceIterator() /// /// /// - public bool Consume(ref Span key) + public bool Consume(ReadOnlySpan key) { // Check if key is within the current processing window only if _copyOption is set // in order to skip keys that have been send over to target node but not deleted locally @@ -165,13 +152,13 @@ public bool Consume(ref Span key) } // Create ArgSlice and check if there is enough space to copy current key - var keySlice = new ArgSlice(currPtr, key.Length); + var keySlice = PinnedSpanByte.FromPinnedPointer(currPtr, key.Length); if (currPtr + keySlice.Length > endPtr) return false; // Copy key to buffer and add it to migrate session dictionary key.CopyTo(keySlice.Span); - if (!session.AddKey(ref keySlice)) + if (!session.AddKey(keySlice)) throw new GarnetException("Failed to add migrating key to working set!"); diff --git a/libs/cluster/Server/Migration/MigrationManager.cs b/libs/cluster/Server/Migration/MigrationManager.cs index b57a8b88be1..7fa7def4e9a 100644 --- a/libs/cluster/Server/Migration/MigrationManager.cs +++ b/libs/cluster/Server/Migration/MigrationManager.cs @@ -4,8 +4,8 @@ using System.Collections.Generic; using System.Runtime.CompilerServices; using Garnet.common; -using Garnet.server; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.cluster { @@ -141,7 +141,7 @@ public bool TryRemoveMigrationTask(string targetNodeId) /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CanAccessKey(ref ArgSlice key, int slot, bool readOnly) - => migrationTaskStore.CanAccessKey(ref key, slot, readOnly); + public bool CanAccessKey(PinnedSpanByte key, int slot, bool readOnly) + => migrationTaskStore.CanAccessKey(key, slot, readOnly); } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index de03c000ab8..b8fcbd82dfb 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -5,7 +5,6 @@ using System.Threading.Tasks; using Garnet.common; using Microsoft.Extensions.Logging; -using Tsavorite.core; namespace Garnet.cluster { @@ -82,30 +81,13 @@ public void SetClusterSyncHeader(bool isMainStore) } /// - /// Try write main store key value pair + /// Try to write the span of an entire record. /// - /// - /// - /// - /// - public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out Task task) - { - WaitForFlush().GetAwaiter().GetResult(); - return AofSyncTask.garnetClient.TryWriteKeyValueSpanByte(ref key, ref value, out task); - } - - /// - /// Try write object store key value pair - /// - /// - /// - /// - /// /// - public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, out Task task) + public bool TryWriteRecordSpan(ReadOnlySpan recordSpan, out Task task) { WaitForFlush().GetAwaiter().GetResult(); - return AofSyncTask.garnetClient.TryWriteKeyValueByteArray(key, value, expiration, out task); + return AofSyncTask.garnetClient.TryWriteRecordSpan(recordSpan, out task); } /// diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index 1e1319ddee4..67f4b782038 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Buffers; using System.Threading; using Garnet.server; using Microsoft.Extensions.Logging; @@ -19,6 +20,11 @@ internal sealed unsafe class SnapshotIteratorManager public MainStoreSnapshotIterator mainStoreSnapshotIterator; public ObjectStoreSnapshotIterator objectStoreSnapshotIterator; + // For serialization from LogRecord to DiskLogRecord + SpanByteAndMemory serializationOutput; + GarnetObjectSerializer valueObjectSerializer; + MemoryPool memoryPool; + readonly ReplicaSyncSession[] sessions; readonly int numSessions; @@ -47,6 +53,9 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ca mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); if (!replicationSyncManager.ClusterProvider.serverOptions.DisableObjects) objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this); + + memoryPool = MemoryPool.Shared; + valueObjectSerializer = new(customCommandManager: default); } /// @@ -80,7 +89,8 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio for (var i = 0; i < numSessions; i++) { - if (!replicationSyncManager.IsActive(i)) continue; + if (!replicationSyncManager.IsActive(i)) + continue; sessions[i].InitializeIterationBuffer(); if (isMainStore) sessions[i].currentStoreVersion = targetVersion; @@ -94,14 +104,20 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio return true; } - public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords) + public bool StringReader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) + where TSourceLogRecord : ISourceLogRecord { if (!firstRead) { + var key = srcLogRecord.Key; + var value = srcLogRecord.ValueSpan; logger?.LogTrace("Start Streaming {key} {value}", key.ToString(), value.ToString()); firstRead = true; } + // Note: We may be sending to multiple replicas, so cannot serialize LogRecords directly to the network buffer + DiskLogRecord.Serialize(ref srcLogRecord, valueSerializer: null, ref serializationOutput, memoryPool); + var needToFlush = false; while (true) { @@ -114,20 +130,22 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe // Write key value pair to network buffer for (var i = 0; i < numSessions; i++) { - if (!replicationSyncManager.IsActive(i)) continue; + if (!replicationSyncManager.IsActive(i)) + continue; // Initialize header if necessary sessions[i].SetClusterSyncHeader(isMainStore: true); // Try to write to network buffer. If failed we need to retry - if (!sessions[i].TryWriteKeyValueSpanByte(ref key, ref value, out var task)) + if (!sessions[i].TryWriteRecordSpan(serializationOutput.MemorySpan, out var task)) { sessions[i].SetFlushTask(task); needToFlush = true; } } - if (!needToFlush) break; + if (!needToFlush) + break; // Wait for flush to complete for all and retry to enqueue previous keyValuePair above replicationSyncManager.WaitForFlush().GetAwaiter().GetResult(); @@ -138,16 +156,21 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe return true; } - public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords) + public bool ObjectReader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) + where TSourceLogRecord : ISourceLogRecord { if (!firstRead) { + var key = srcLogRecord.Key; + var value = srcLogRecord.ValueObject; logger?.LogTrace("Start Streaming {key} {value}", key.ToString(), value.ToString()); firstRead = true; } + // Note: We may be sending to multiple replicas, so cannot serialize LogRecords directly to the network buffer + DiskLogRecord.Serialize(ref srcLogRecord, valueObjectSerializer, ref serializationOutput, memoryPool); + var needToFlush = false; - var objectData = GarnetObjectSerializer.Serialize(value); while (true) { if (cancellationToken.IsCancellationRequested) @@ -159,20 +182,22 @@ public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recor // Write key value pair to network buffer for (var i = 0; i < numSessions; i++) { - if (!replicationSyncManager.IsActive(i)) continue; + if (!replicationSyncManager.IsActive(i)) + continue; // Initialize header if necessary sessions[i].SetClusterSyncHeader(isMainStore: false); // Try to write to network buffer. If failed we need to retry - if (!sessions[i].TryWriteKeyValueByteArray(key, objectData, value.Expiration, out var task)) + if (!sessions[i].TryWriteRecordSpan(serializationOutput.MemorySpan, out var task)) { sessions[i].SetFlushTask(task); needToFlush = true; } } - if (!needToFlush) break; + if (!needToFlush) + break; // Wait for flush to complete for all and retry to enqueue previous keyValuePair above replicationSyncManager.WaitForFlush().GetAwaiter().GetResult(); @@ -187,8 +212,8 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long // Flush remaining data for (var i = 0; i < numSessions; i++) { - if (!replicationSyncManager.IsActive(i)) continue; - sessions[i].SendAndResetIterationBuffer(); + if (replicationSyncManager.IsActive(i)) + sessions[i].SendAndResetIterationBuffer(); } // Wait for flush and response to complete @@ -199,11 +224,13 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long // Reset read marker firstRead = false; + + serializationOutput.Dispose(); } } internal sealed unsafe class MainStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) : - IStreamingSnapshotIteratorFunctions + IStreamingSnapshotIteratorFunctions { readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager; long targetVersion; @@ -214,8 +241,9 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: true); } - public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords) - => snapshotIteratorManager.Reader(ref key, ref value, recordMetadata, numberOfRecords); + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) + where TSourceLogRecord : ISourceLogRecord + => snapshotIteratorManager.StringReader(ref srcLogRecord, recordMetadata, numberOfRecords); public void OnException(Exception exception, long numberOfRecords) => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(MainStoreSnapshotIterator)}"); @@ -225,7 +253,7 @@ public void OnStop(bool completed, long numberOfRecords) } internal sealed unsafe class ObjectStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) : - IStreamingSnapshotIteratorFunctions + IStreamingSnapshotIteratorFunctions { readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager; long targetVersion; @@ -236,8 +264,9 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: false); } - public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords) - => snapshotIteratorManager.Reader(ref key, ref value, recordMetadata, numberOfRecords); + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) + where TSourceLogRecord : ISourceLogRecord + => snapshotIteratorManager.ObjectReader(ref srcLogRecord, recordMetadata, numberOfRecords); public void OnException(Exception exception, long numberOfRecords) => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(ObjectStoreSnapshotIterator)}"); diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs index 14f9d2ad908..f078305e9e0 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs @@ -12,7 +12,7 @@ namespace Garnet.cluster { internal sealed partial class ReplicationManager : IDisposable { - // Must be the same as the TsavoriteLog start address of allocator + // Must be the same as the TsavoriteAof start address of allocator public static readonly long kFirstValidAofAddress = 64; readonly AofTaskStore aofTaskStore; diff --git a/libs/cluster/Session/ClusterKeyIterationFunctions.cs b/libs/cluster/Session/ClusterKeyIterationFunctions.cs index 54d91d6cd3d..020b79426bf 100644 --- a/libs/cluster/Session/ClusterKeyIterationFunctions.cs +++ b/libs/cluster/Session/ClusterKeyIterationFunctions.cs @@ -22,7 +22,7 @@ internal class KeyIterationInfo internal KeyIterationInfo(int slot) => this.slot = slot; } - internal sealed class MainStoreCountKeys : IScanIteratorFunctions + internal sealed class MainStoreCountKeys : IScanIteratorFunctions { private readonly KeyIterationInfo info; // This must be a class as it is passed through pending IO operations @@ -32,21 +32,21 @@ internal sealed class MainStoreCountKeys : IScanIteratorFunctions info = new(slot); - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here - if (HashSlotUtils.HashSlot(ref key) == Slot && !Expired(ref value)) + if (HashSlotUtils.HashSlot(srcLogRecord.Key) == Slot && !Expired(ref srcLogRecord)) KeyCount++; return true; } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + 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 ObjectStoreCountKeys : IScanIteratorFunctions + internal sealed class ObjectStoreCountKeys : IScanIteratorFunctions { private readonly KeyIterationInfo info; // This must be a class as it is passed through pending IO operations @@ -56,24 +56,21 @@ internal sealed class ObjectStoreCountKeys : IScanIteratorFunctions info = new(slot); - public bool SingleReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here , out CursorRecordResult cursorRecordResult - fixed (byte* keyPtr = key) - { - if (HashSlotUtils.HashSlot(keyPtr, key.Length) == Slot && !Expired(ref value)) - KeyCount++; - } + if (HashSlotUtils.HashSlot(srcLogRecord.Key) == Slot && !Expired(ref srcLogRecord)) + KeyCount++; return true; } - public bool ConcurrentReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } } - internal readonly struct MainStoreGetKeysInSlot : IScanIteratorFunctions + internal readonly struct MainStoreGetKeysInSlot : IScanIteratorFunctions { readonly List keys; readonly int slot, maxKeyCount; @@ -85,21 +82,22 @@ internal MainStoreGetKeysInSlot(List keys, int slot, int maxKeyCount) this.maxKeyCount = maxKeyCount; } - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here, out CursorRecordResult cursorRecordResult - if (HashSlotUtils.HashSlot(ref key) == slot && !Expired(ref value)) - keys.Add(key.ToByteArray()); + var key = srcLogRecord.Key; + if (HashSlotUtils.HashSlot(key) == slot && !Expired(ref srcLogRecord)) + keys.Add(key.ToArray()); return keys.Count < maxKeyCount; } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } } - internal readonly struct ObjectStoreGetKeysInSlot : IScanIteratorFunctions + internal readonly struct ObjectStoreGetKeysInSlot : IScanIteratorFunctions { readonly List keys; readonly int slot; @@ -110,18 +108,16 @@ internal ObjectStoreGetKeysInSlot(List keys, int slot) this.slot = slot; } - public bool SingleReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; // default; not used here - fixed (byte* keyPtr = key) - { - if (HashSlotUtils.HashSlot(keyPtr, key.Length) == slot && !Expired(ref value)) - keys.Add(key); - } + var key = srcLogRecord.Key; + if (HashSlotUtils.HashSlot(key) == slot && !Expired(ref srcLogRecord)) + keys.Add(key.ToArray()); return true; } - public bool ConcurrentReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs index facbad2ac60..4984270eb2f 100644 --- a/libs/cluster/Session/ClusterSession.cs +++ b/libs/cluster/Session/ClusterSession.cs @@ -12,12 +12,12 @@ namespace Garnet.cluster { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; internal sealed unsafe partial class ClusterSession : IClusterSession { diff --git a/libs/cluster/Session/MigrateCommand.cs b/libs/cluster/Session/MigrateCommand.cs index 62a7d48d120..acf4974278d 100644 --- a/libs/cluster/Session/MigrateCommand.cs +++ b/libs/cluster/Session/MigrateCommand.cs @@ -13,9 +13,9 @@ namespace Garnet.cluster { internal sealed unsafe partial class ClusterSession : IClusterSession { - public static bool Expired(ref SpanByte value) => value.MetadataSize > 0 && value.ExtraMetadata < DateTimeOffset.UtcNow.Ticks; - - public static bool Expired(ref IGarnetObject value) => value.Expiration != 0 && value.Expiration < DateTimeOffset.UtcNow.Ticks; + public static bool Expired(ref TSourceLogRecord logRecord) + where TSourceLogRecord : ISourceLogRecord + => logRecord.Info.HasExpiration && logRecord.Expiration < DateTimeOffset.UtcNow.Ticks; internal enum MigrateCmdParseState : byte { @@ -116,7 +116,7 @@ private bool TryMIGRATE(out bool invalidParameters) { transferOption = TransferOption.KEYS; keys = new(); - _ = keys.TryAdd(ref keySlice, KeyMigrationStatus.QUEUED); + _ = keys.TryAdd(keySlice, KeyMigrationStatus.QUEUED); } var currTokenIdx = 5; @@ -148,13 +148,12 @@ private bool TryMIGRATE(out bool invalidParameters) while (currTokenIdx < parseState.Count) { var currKeySlice = parseState.GetArgSliceByRef(currTokenIdx++); - var sbKey = currKeySlice.SpanByte; // Skip if previous error encountered if (pstate != MigrateCmdParseState.SUCCESS) continue; // Check if all keys are local R/W because we migrate keys and need to be able to delete them - var slot = HashSlotUtils.HashSlot(sbKey.ToPointer(), sbKey.Length); + var slot = HashSlotUtils.HashSlot(currKeySlice.ToPointer(), currKeySlice.Length); if (!current.IsLocal(slot, readWriteSession: false)) { pstate = MigrateCmdParseState.SLOTNOTLOCAL; @@ -176,7 +175,7 @@ private bool TryMIGRATE(out bool invalidParameters) } // Add pointer of current parsed key - if (!keys.TryAdd(ref currKeySlice, KeyMigrationStatus.QUEUED)) + if (!keys.TryAdd(currKeySlice, KeyMigrationStatus.QUEUED)) { logger?.LogWarning("Failed to add {key}", Encoding.ASCII.GetString(keySlice.ReadOnlySpan)); pstate = MigrateCmdParseState.FAILEDTOADDKEY; diff --git a/libs/cluster/Session/RespClusterBasicCommands.cs b/libs/cluster/Session/RespClusterBasicCommands.cs index 4db65c4e243..2097542f52f 100644 --- a/libs/cluster/Session/RespClusterBasicCommands.cs +++ b/libs/cluster/Session/RespClusterBasicCommands.cs @@ -366,7 +366,7 @@ private bool NetworkClusterGossip(out bool invalidParameters) gossipWithMeet = true; } - var gossipMessage = parseState.GetArgSliceByRef(currTokenIdx).SpanByte.ToByteArray(); + var gossipMessage = parseState.GetArgSliceByRef(currTokenIdx).ToArray(); clusterProvider.clusterManager.gossipStats.UpdateGossipBytesRecv(gossipMessage.Length); var current = clusterProvider.clusterManager.CurrentConfig; diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs index 3e789d0fcae..6d68ef12cf5 100644 --- a/libs/cluster/Session/RespClusterMigrateCommands.cs +++ b/libs/cluster/Session/RespClusterMigrateCommands.cs @@ -52,82 +52,73 @@ private bool NetworkClusterMigrate(out bool invalidParameters) var replaceSpan = parseState.GetArgSliceByRef(1).ReadOnlySpan; var storeTypeSpan = parseState.GetArgSliceByRef(2).ReadOnlySpan; - var payload = parseState.GetArgSliceByRef(3).SpanByte; + var payload = parseState.GetArgSliceByRef(3); var payloadPtr = payload.ToPointer(); - var lastParam = parseState.GetArgSliceByRef(parseState.Count - 1).SpanByte; + var lastParam = parseState.GetArgSliceByRef(parseState.Count - 1); var payloadEndPtr = lastParam.ToPointer() + lastParam.Length; var replaceOption = replaceSpan.EqualsUpperCaseSpanIgnoringCase("T"u8); var currentConfig = clusterProvider.clusterManager.CurrentConfig; - byte migrateState = 0; + var migrateError = false; if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("SSTORE"u8)) { var keyCount = *(int*)payloadPtr; payloadPtr += 4; - var i = 0; - TrackImportProgress(keyCount, isMainStore: true, keyCount == 0); - while (i < keyCount) + + for (var ii = 0; ii < keyCount; ++ii) { - ref var key = ref SpanByte.Reinterpret(payloadPtr); - payloadPtr += key.TotalSize; - ref var value = ref SpanByte.Reinterpret(payloadPtr); - payloadPtr += value.TotalSize; + if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) + return false; - // An error has occurred - if (migrateState > 0) - { - i++; + // If an error has occurred, continue to drain all records + if (migrateError) continue; - } - var slot = HashSlotUtils.HashSlot(ref key); + var diskLogRecord = new DiskLogRecord(startAddress, length); + var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state { - migrateState = 1; - i++; + migrateError = true; continue; } // Set if key replace flag is set or key does not exist - var keySlice = new ArgSlice(key.ToPointer(), key.Length); - if (replaceOption || !Exists(ref keySlice)) - _ = basicGarnetApi.SET(ref key, ref value); - i++; + if (replaceOption || !Exists(PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key))) + _ = basicGarnetApi.SET(ref diskLogRecord, StoreType.Main); } } else if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("OSTORE"u8)) { var keyCount = *(int*)payloadPtr; payloadPtr += 4; - var i = 0; TrackImportProgress(keyCount, isMainStore: false, keyCount == 0); - while (i < keyCount) + + for (var ii = 0; ii < keyCount; ++ii) { - if (!RespReadUtils.TryReadSerializedData(out var key, out var data, out var expiration, ref payloadPtr, payloadEndPtr)) + if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) return false; - // An error has occurred - if (migrateState > 0) + // If an error has occurred, continue to drain all records + if (migrateError) continue; - var slot = HashSlotUtils.HashSlot(key); + var diskLogRecord = new DiskLogRecord(startAddress, length); + var slot = HashSlotUtils.HashSlot(diskLogRecord.Key); if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state { - migrateState = 1; + migrateError = true; continue; } - var value = clusterProvider.storeWrapper.GarnetObjectSerializer.Deserialize(data); - value.Expiration = expiration; - // Set if key replace flag is set or key does not exist - if (replaceOption || !CheckIfKeyExists(key)) - _ = basicGarnetApi.SET(key, value); - - i++; + if (replaceOption || !Exists(PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key))) + { + _ = diskLogRecord.DeserializeValueObject(clusterProvider.storeWrapper.GarnetObjectSerializer); + _ = basicGarnetApi.SET(ref diskLogRecord, StoreType.Object); + } } } else @@ -135,7 +126,7 @@ private bool NetworkClusterMigrate(out bool invalidParameters) throw new Exception("CLUSTER MIGRATE STORE TYPE ERROR!"); } - if (migrateState == 1) + if (migrateError) { logger?.LogError("{errorMsg}", Encoding.ASCII.GetString(CmdStrings.RESP_ERR_GENERIC_NOT_IN_IMPORTING_STATE)); while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_NOT_IN_IMPORTING_STATE, ref dcurr, dend)) diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index 6bab1fdce62..6628e585257 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -132,7 +132,7 @@ private bool NetworkClusterAOFSync(out bool invalidParameters) if (clusterProvider.serverOptions.EnableAOF) { - clusterProvider.replicationManager.TryAddReplicationTask(nodeId, nextAddress, out var aofSyncTaskInfo); + _ = clusterProvider.replicationManager.TryAddReplicationTask(nodeId, nextAddress, out var aofSyncTaskInfo); if (!clusterProvider.replicationManager.TryConnectToReplica(nodeId, nextAddress, aofSyncTaskInfo, out var errorMessage)) { while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) @@ -179,7 +179,7 @@ private bool NetworkClusterAppendLog(out bool invalidParameters) return true; } - var sbRecord = parseState.GetArgSliceByRef(4).SpanByte; + var sbRecord = parseState.GetArgSliceByRef(4); var currentConfig = clusterProvider.clusterManager.CurrentConfig; var localRole = currentConfig.LocalNodeRole; @@ -219,7 +219,7 @@ private bool NetworkClusterInitiateReplicaSync(out bool invalidParameters) var replicaNodeId = parseState.GetString(0); var replicaAssignedPrimaryId = parseState.GetString(1); - var checkpointEntryBytes = parseState.GetArgSliceByRef(2).SpanByte.ToByteArray(); + var checkpointEntryBytes = parseState.GetArgSliceByRef(2).ToArray(); if (!parseState.TryGetLong(3, out var replicaAofBeginAddress) || !parseState.TryGetLong(4, out var replicaAofTailAddress)) @@ -276,7 +276,7 @@ private bool NetworkClusterSendCheckpointMetadata(out bool invalidParameters) return true; } - var checkpointMetadata = parseState.GetArgSliceByRef(2).SpanByte.ToByteArray(); + var checkpointMetadata = parseState.GetArgSliceByRef(2).ToArray(); var fileToken = new Guid(fileTokenBytes); var fileType = (CheckpointFileType)fileTypeInt; @@ -353,7 +353,7 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters) } var primaryReplicaId = parseState.GetString(3); - var checkpointEntryBytes = parseState.GetArgSliceByRef(4).SpanByte.ToByteArray(); + var checkpointEntryBytes = parseState.GetArgSliceByRef(4).ToArray(); if (!parseState.TryGetLong(5, out var beginAddress) || !parseState.TryGetLong(6, out var tailAddress)) @@ -404,7 +404,7 @@ private bool NetworkClusterAttachSync(out bool invalidParameters) return true; } - var checkpointEntryBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var checkpointEntryBytes = parseState.GetArgSliceByRef(0).ToArray(); var syncMetadata = SyncMetadata.FromByteArray(checkpointEntryBytes); ReadOnlySpan errorMessage = default; @@ -446,39 +446,38 @@ private bool NetworkClusterSync(out bool invalidParameters) var primaryNodeId = parseState.GetString(0); var storeTypeSpan = parseState.GetArgSliceByRef(1).ReadOnlySpan; - var payload = parseState.GetArgSliceByRef(2).SpanByte; + var payload = parseState.GetArgSliceByRef(2); var payloadPtr = payload.ToPointer(); - var lastParam = parseState.GetArgSliceByRef(parseState.Count - 1).SpanByte; + var lastParam = parseState.GetArgSliceByRef(parseState.Count - 1); var payloadEndPtr = lastParam.ToPointer() + lastParam.Length; - var keyValuePairCount = *(int*)payloadPtr; + var recordCount = *(int*)payloadPtr; var i = 0; payloadPtr += 4; if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("SSTORE"u8)) { - TrackImportProgress(keyValuePairCount, isMainStore: true, keyValuePairCount == 0); - while (i < keyValuePairCount) + TrackImportProgress(recordCount, isMainStore: true, recordCount == 0); + while (i < recordCount) { - ref var key = ref SpanByte.Reinterpret(payloadPtr); - payloadPtr += key.TotalSize; - ref var value = ref SpanByte.Reinterpret(payloadPtr); - payloadPtr += value.TotalSize; + if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) + return false; - _ = basicGarnetApi.SET(ref key, ref value); + var diskLogRecord = new DiskLogRecord(startAddress, length); + _ = basicGarnetApi.SET(ref diskLogRecord, StoreType.Main); i++; } } else if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("OSTORE"u8)) { - TrackImportProgress(keyValuePairCount, isMainStore: false, keyValuePairCount == 0); - while (i < keyValuePairCount) + TrackImportProgress(recordCount, isMainStore: false, recordCount == 0); + while (i < recordCount) { - if (!RespReadUtils.TryReadSerializedData(out var key, out var data, out var expiration, ref payloadPtr, payloadEndPtr)) + if (!RespReadUtils.TryReadSerializedRecord(out var startAddress, out var length, ref payloadPtr, payloadEndPtr)) return false; - var value = clusterProvider.storeWrapper.GarnetObjectSerializer.Deserialize(data); - value.Expiration = expiration; - _ = basicGarnetApi.SET(key, value); + var diskLogRecord = new DiskLogRecord(startAddress, length); + _ = diskLogRecord.DeserializeValueObject(clusterProvider.storeWrapper.GarnetObjectSerializer); + _ = basicGarnetApi.SET(ref diskLogRecord, StoreType.Object); i++; } } diff --git a/libs/cluster/Session/RespClusterSlotManagementCommands.cs b/libs/cluster/Session/RespClusterSlotManagementCommands.cs index 97868251827..692f37ba8b9 100644 --- a/libs/cluster/Session/RespClusterSlotManagementCommands.cs +++ b/libs/cluster/Session/RespClusterSlotManagementCommands.cs @@ -405,7 +405,7 @@ private bool NetworkClusterKeySlot(out bool invalidParameters) return true; } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var sbKey = parseState.GetArgSliceByRef(0); var keyPtr = sbKey.ToPointer(); var keySize = sbKey.Length; diff --git a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs index 0416c064d43..bcb34cd55f1 100644 --- a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs +++ b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs @@ -4,33 +4,26 @@ using System; using System.Runtime.CompilerServices; using System.Threading; +using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet.cluster { internal sealed unsafe partial class ClusterSession : IClusterSession { [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool Exists(ref ArgSlice keySlice) + private bool Exists(PinnedSpanByte keySlice) => basicGarnetApi.EXISTS(keySlice, StoreType.All) == GarnetStatus.OK; - private bool CheckIfKeyExists(byte[] key) - { - fixed (byte* keyPtr = key) - { - var keySlice = new ArgSlice(keyPtr, key.Length); - return Exists(ref keySlice); - } - } - - private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref ArgSlice keySlice, bool readOnly, byte SessionAsking, int slot = -1) + private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref PinnedSpanByte keySlice, bool readOnly, byte SessionAsking, int slot = -1) { return readOnly ? SingleKeyReadSlotVerify(ref config, ref keySlice) : SingleKeyReadWriteSlotVerify(ref config, ref keySlice); [MethodImpl(MethodImplOptions.AggressiveInlining)] - ClusterSlotVerificationResult SingleKeyReadSlotVerify(ref ClusterConfig config, ref ArgSlice keySlice) + ClusterSlotVerificationResult SingleKeyReadSlotVerify(ref ClusterConfig config, ref PinnedSpanByte keySlice) { - var _slot = slot == -1 ? ArgSliceUtils.HashSlot(ref keySlice) : (ushort)slot; + var _slot = slot == -1 ? HashSlotUtils.HashSlot(keySlice) : (ushort)slot; var IsLocal = config.IsLocal(_slot); var state = config.GetState(_slot); @@ -69,9 +62,9 @@ ClusterSlotVerificationResult SingleKeyReadSlotVerify(ref ClusterConfig config, } [MethodImpl(MethodImplOptions.AggressiveInlining)] - ClusterSlotVerificationResult SingleKeyReadWriteSlotVerify(ref ClusterConfig config, ref ArgSlice keySlice) + ClusterSlotVerificationResult SingleKeyReadWriteSlotVerify(ref ClusterConfig config, ref PinnedSpanByte keySlice) { - var _slot = slot == -1 ? ArgSliceUtils.HashSlot(ref keySlice) : (ushort)slot; + var _slot = slot == -1 ? HashSlotUtils.HashSlot(keySlice) : (ushort)slot; var IsLocal = config.IsLocal(_slot, readWriteSession: readWriteSession); var state = config.GetState(_slot); @@ -110,30 +103,30 @@ ClusterSlotVerificationResult SingleKeyReadWriteSlotVerify(ref ClusterConfig con } } - bool CanOperateOnKey(ref ArgSlice key, int slot, bool readOnly) + bool CanOperateOnKey(ref PinnedSpanByte key, int slot, bool readOnly) { // For both read and read/write ops we need to ensure that key will not be removed // while we try to operate on it so we will delay the corresponding operation // as long as the key is being actively migrated - while (!clusterProvider.migrationManager.CanAccessKey(ref key, slot, readOnly)) + while (!clusterProvider.migrationManager.CanAccessKey(key, slot, readOnly)) { ReleaseCurrentEpoch(); Thread.Yield(); AcquireCurrentEpoch(); } - return Exists(ref key); + return Exists(key); } } - ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Span keys, bool readOnly, byte sessionAsking, int count) + ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Span keys, bool readOnly, byte sessionAsking, int count) { var _end = count < 0 ? keys.Length : count; - var slot = ArgSliceUtils.HashSlot(ref keys[0]); + var slot = HashSlotUtils.HashSlot(keys[0]); var verifyResult = SingleKeySlotVerify(ref config, ref keys[0], readOnly, sessionAsking, slot); for (var i = 1; i < _end; i++) { - var _slot = ArgSliceUtils.HashSlot(ref keys[i]); + var _slot = HashSlotUtils.HashSlot(keys[i]); var _verifyResult = SingleKeySlotVerify(ref config, ref keys[i], readOnly, sessionAsking, _slot); // Check if slot changes between keys @@ -151,7 +144,7 @@ ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Span< ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref SessionParseState parseState, ref ClusterSlotVerificationInput csvi) { ref var key = ref parseState.GetArgSliceByRef(csvi.firstKey); - var slot = ArgSliceUtils.HashSlot(ref key); + var slot = HashSlotUtils.HashSlot(key); var verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, slot); var secondKey = csvi.firstKey + csvi.step; @@ -160,7 +153,7 @@ ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Sessi if (csvi.keyNumOffset == i) continue; key = ref parseState.GetArgSliceByRef(i); - var _slot = ArgSliceUtils.HashSlot(ref key); + var _slot = HashSlotUtils.HashSlot(key); var _verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, _slot); // Check if slot changes between keys diff --git a/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs b/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs index 3fe36867e9c..3ffbe5b14bd 100644 --- a/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs +++ b/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet.cluster { @@ -28,7 +29,7 @@ public void ResetCachedSlotVerificationResult() /// /// /// - public bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking) + public bool NetworkIterativeSlotVerify(PinnedSpanByte keySlice, bool readOnly, byte SessionAsking) { ClusterSlotVerificationResult verifyResult; diff --git a/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs index af69ed8d2b2..c4b4687405a 100644 --- a/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs +++ b/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs @@ -7,6 +7,7 @@ using Garnet.common; using Garnet.server; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.cluster { @@ -95,7 +96,7 @@ private void WriteClusterSlotVerificationMessage(ClusterConfig config, ClusterSl /// /// /// - public bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte sessionAsking, ref byte* dcurr, ref byte* dend, int count = -1) + public bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte sessionAsking, ref byte* dcurr, ref byte* dend, int count = -1) { // If cluster is not enabled or a transaction is running skip slot check if (!clusterProvider.serverOptions.EnableCluster || txnManager.state == TxnState.Running) return false; diff --git a/libs/common/HashSlotUtils.cs b/libs/common/HashSlotUtils.cs index f1811ce3a7e..31feff3d942 100644 --- a/libs/common/HashSlotUtils.cs +++ b/libs/common/HashSlotUtils.cs @@ -71,19 +71,24 @@ internal static unsafe ushort Hash(byte* data, int len) return result; } + /// + /// Compute hash slot from the given ArgSlice + /// + public static unsafe ushort HashSlot(PinnedSpanByte argSlice) + => HashSlot(argSlice.ToPointer(), argSlice.Length); + /// /// Compute hash slot from the given SpanByte /// - /// - /// - public static unsafe ushort HashSlot(ref SpanByte key) - => HashSlot(key.ToPointer(), key.LengthWithoutMetadata); + public static unsafe ushort HashSlot(ReadOnlySpan key) + { + fixed (byte* keyPtr = key) + return HashSlot(keyPtr, key.Length); + } /// /// Compute hash slot of given data /// - /// - /// public static unsafe ushort HashSlot(Span key) { fixed (byte* keyPtr = key) diff --git a/libs/common/Networking/GarnetTcpNetworkSender.cs b/libs/common/Networking/GarnetTcpNetworkSender.cs index 29e71b39893..8c3d77a38d4 100644 --- a/libs/common/Networking/GarnetTcpNetworkSender.cs +++ b/libs/common/Networking/GarnetTcpNetworkSender.cs @@ -202,7 +202,7 @@ void ReturnBuffer(GarnetSaeaBuffer buffer) { if (responseObject != null) return responseObject.buffer.entryPtr + responseObject.buffer.entry.Length; - return base.GetResponseObjectHead(); + return base.GetResponseObjectTail(); } /// @@ -214,6 +214,7 @@ public override bool SendResponse(int offset, int size) responseObject = null; try { + // If this does not throw, _r is ReturnBuffer()ed when it completes. Send(socket, _r, offset, size); } catch diff --git a/libs/common/RespReadUtils.cs b/libs/common/RespReadUtils.cs index 8529366aea9..7bfff9fa8b7 100644 --- a/libs/common/RespReadUtils.cs +++ b/libs/common/RespReadUtils.cs @@ -1217,84 +1217,23 @@ public static bool TryReadAsSpan(out ReadOnlySpan result, ref byte* ptr, b /// /// Read serialized data for migration - /// - public static bool TryReadSerializedSpanByte(ref byte* keyPtr, ref byte keyMetaDataSize, ref byte* valPtr, ref byte valMetaDataSize, ref byte* ptr, byte* end) + /// + public static bool TryReadSerializedRecord(out long recordStartAddress, out int recordLength, ref byte* ptr, byte* end) { - //1. safe read ksize - if (ptr + sizeof(int) > end) - return false; - var ksize = *(int*)ptr; - ptr += sizeof(int); + recordStartAddress = 0; + recordLength = 0; - //2. safe read key bytes - if (ptr + ksize + 1 > end) - return false; - keyPtr = ptr - sizeof(int); - ptr += ksize; - keyMetaDataSize = *ptr++; - - //3. safe read vsize - if (ptr + 4 > end) + //1. safe read recordSize + if (ptr + sizeof(int) > end) return false; - var vsize = *(int*)ptr; + recordLength = *(int*)ptr; ptr += sizeof(int); - //4. safe read value bytes - if (ptr + vsize + 1 > end) - return false; - valPtr = ptr - sizeof(int); - ptr += vsize; - valMetaDataSize = *ptr++; - - return true; - } - - /// - /// Read serialized data for migration - /// - public static bool TryReadSerializedData(out byte[] key, out byte[] value, out long expiration, ref byte* ptr, byte* end) - { - expiration = -1; - key = null; - value = null; - - //1. safe read ksize - if (ptr + 4 > end) - return false; - var keyLen = *(int*)ptr; - ptr += 4; - //2. safe read keyPtr - if (ptr + keyLen > end) - return false; - var keyPtr = ptr; - ptr += keyLen; - - //3. safe read vsize - if (ptr + 4 > end) - return false; - var valLen = *(int*)ptr; - ptr += 4; - - //4. safe read valPtr - if (ptr + valLen > end) + if (ptr + recordLength > end) return false; - var valPtr = ptr; - ptr += valLen; - - //5. safe read expiration info - if (ptr + 8 > end) - return false; - expiration = *(long*)ptr; - ptr += 8; - - key = new byte[keyLen]; - value = new byte[valLen]; - fixed (byte* kPtr = key) - Buffer.MemoryCopy(keyPtr, kPtr, keyLen, keyLen); - fixed (byte* vPtr = value) - Buffer.MemoryCopy(valPtr, vPtr, valLen, valLen); - + recordStartAddress = (long)ptr; + ptr += recordLength; return true; } } diff --git a/libs/common/RespWriteUtils.cs b/libs/common/RespWriteUtils.cs index 74eff5db4e3..63e710bd09a 100644 --- a/libs/common/RespWriteUtils.cs +++ b/libs/common/RespWriteUtils.cs @@ -729,13 +729,13 @@ public static bool TryWriteFalse(ref byte* curr, byte* end) public static void WriteEtagValArray(long etag, ref ReadOnlySpan value, ref byte* curr, byte* end, bool writeDirect) { // Writes a Resp encoded Array of Integer for ETAG as first element, and bulk string for value as second element - RespWriteUtils.TryWriteArrayLength(2, ref curr, end); - RespWriteUtils.TryWriteInt64(etag, ref curr, end); + TryWriteArrayLength(2, ref curr, end); + TryWriteInt64(etag, ref curr, end); if (writeDirect) - RespWriteUtils.TryWriteDirect(value, ref curr, end); + TryWriteDirect(value, ref curr, end); else - RespWriteUtils.TryWriteBulkString(value, ref curr, end); + TryWriteBulkString(value, ref curr, end); } /// diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index 69050d1a7c7..5fdd7edbf1c 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -20,11 +20,11 @@ namespace Garnet { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Implementation Garnet server @@ -48,8 +48,8 @@ static string GetVersion() private readonly GarnetServerOptions opts; private IGarnetServer[] servers; private SubscribeBroker subscribeBroker; - private KVSettings kvSettings; - private KVSettings objKvSettings; + private KVSettings kvSettings; + private KVSettings objKvSettings; private INamedDeviceFactory logFactory; private MemoryLogger initLogger; private ILogger logger; @@ -233,7 +233,7 @@ private void InitializeServer() CreateDatabase(dbId, opts, clusterFactory, customCommandManager); if (!opts.DisablePubSub) - subscribeBroker = new SubscribeBroker(null, opts.PubSubPageSizeBytes(), opts.SubscriberRefreshFrequencyMs, true, logger); + subscribeBroker = new SubscribeBroker(null, opts.PubSubPageSizeBytes(), opts.SubscriberRefreshFrequencyMs, startFresh: true, logger); logger?.LogTrace("TLS is {tlsEnabled}", opts.TlsOptions == null ? "disabled" : "enabled"); @@ -326,7 +326,7 @@ private void LoadModules(CustomCommandManager customCommandManager) } } - private TsavoriteKV CreateMainStore(int dbId, IClusterFactory clusterFactory, + private TsavoriteKV CreateMainStore(int dbId, IClusterFactory clusterFactory, out LightEpoch epoch, out StateMachineDriver stateMachineDriver) { epoch = new LightEpoch(); @@ -344,12 +344,12 @@ private TsavoriteKV clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: true, logger) : new DeviceLogCommitCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true); - return new TsavoriteKV(kvSettings - , StoreFunctions.Create() + return new(kvSettings + , StoreFunctions.Create() , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); } - private TsavoriteKV CreateObjectStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager, + private TsavoriteKV CreateObjectStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager, LightEpoch epoch, StateMachineDriver stateMachineDriver, out CacheSizeTracker objectStoreSizeTracker) { objectStoreSizeTracker = null; @@ -369,12 +369,10 @@ private TsavoriteKV( - objKvSettings, - StoreFunctions.Create(new ByteArrayKeyComparer(), - () => new ByteArrayBinaryObjectSerializer(), - () => new GarnetObjectSerializer(customCommandManager)), - (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); + var objStore = new TsavoriteKV(objKvSettings + , StoreFunctions.Create(new SpanByteComparer(), + () => new GarnetObjectSerializer(customCommandManager)) + , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); if (objHeapMemorySize > 0 || objReadCacheHeapMemorySize > 0) objectStoreSizeTracker = new CacheSizeTracker(objStore, objKvSettings, objHeapMemorySize, objReadCacheHeapMemorySize, @@ -398,7 +396,8 @@ private TsavoriteKV _memoryLog = new(); - public IDisposable BeginScope(TState state) - { - return null; - } + public IDisposable BeginScope(TState state) => null; - public bool IsEnabled(LogLevel logLevel) - { - return true; - } + public bool IsEnabled(LogLevel logLevel) => true; public void Log(LogLevel logLevel, EventId eventId, TState state, Exception exception, Func formatter) - { - this._memoryLog.Add((logLevel, exception, formatter(state, exception))); - } + => this._memoryLog.Add((logLevel, exception, formatter(state, exception))); /// /// Flushes logger entries into a destination logger. @@ -63,9 +55,6 @@ public void Dispose() internal static class LoggingBuilderExtensions { - public static ILoggingBuilder AddMemory(this ILoggingBuilder builder) - { - return builder.AddProvider(new MemoryLoggerProvider()); - } + public static ILoggingBuilder AddMemory(this ILoggingBuilder builder) => builder.AddProvider(new MemoryLoggerProvider()); } } \ No newline at end of file diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index c24396f7363..d9c4804a35f 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -5,7 +5,6 @@ using System.Buffers; using System.Collections.Generic; using System.Diagnostics; -using System.Runtime.CompilerServices; using System.Runtime.InteropServices; using Garnet.common; using Garnet.networking; @@ -14,11 +13,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information @@ -38,12 +37,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; @@ -77,7 +76,7 @@ public AofProcessor( objectStoreInput.parseState = parseState; customProcInput.parseState = parseState; - inflightTxns = new Dictionary>(); + inflightTxns = []; buffer = new byte[BufferSizeUtils.ServerBufferSize(new MaxSizeSettings())]; handle = GCHandle.Alloc(buffer, GCHandleType.Pinned); bufferPtr = (byte*)handle.AddrOfPinnedObject(); @@ -391,14 +390,14 @@ void RunStoredProc(byte id, CustomProcedureInput customProcInput, byte* ptr) respServerSession.RunTransactionProc(id, ref customProcInput, ref output); } - static void StoreUpsert(BasicContext basicContext, + static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr) { var curr = ptr + sizeof(AofHeader); - ref var key = ref Unsafe.AsRef(curr); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); curr += key.TotalSize; - ref var value = ref Unsafe.AsRef(curr); + var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); curr += value.TotalSize; // Reconstructing RawStringInput @@ -407,79 +406,77 @@ static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr) + static void StoreRMW(BasicContext basicContext, RawStringInput storeInput, byte* ptr) { var curr = ptr + sizeof(AofHeader); - ref var key = ref Unsafe.AsRef(curr); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); curr += key.TotalSize; // Reconstructing RawStringInput // input - storeInput.DeserializeFrom(curr); + _ = storeInput.DeserializeFrom(curr); var pbOutput = stackalloc byte[32]; - var output = new SpanByteAndMemory(pbOutput, 32); + var output = SpanByteAndMemory.FromPinnedPointer(pbOutput, 32); - if (basicContext.RMW(ref key, ref storeInput, ref output).IsPending) + if (basicContext.RMW(key.ReadOnlySpan, ref storeInput, ref output).IsPending) basicContext.CompletePending(true); - if (!output.IsSpanByte) - output.Memory.Dispose(); + output.Dispose(); } - static void StoreDelete(BasicContext basicContext, byte* ptr) + static void StoreDelete(BasicContext basicContext, byte* ptr) { - ref var key = ref Unsafe.AsRef(ptr + sizeof(AofHeader)); - basicContext.Delete(ref key); + 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) { - ref var key = ref Unsafe.AsRef(ptr + sizeof(AofHeader)); - var keyB = key.ToByteArray(); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader)); + + var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader) + key.TotalSize); + var valB = garnetObjectSerializer.Deserialize(value.ToArray()); - ref var value = ref Unsafe.AsRef(ptr + sizeof(AofHeader) + key.TotalSize); - var valB = garnetObjectSerializer.Deserialize(value.ToByteArray()); + // input + // TODOMigrate: _ = objectStoreInput.DeserializeFrom(curr); // TODO - need to serialize this as well - var output = new GarnetObjectStoreOutput(new(outputPtr, outputLength)); - basicContext.Upsert(ref keyB, ref valB); + var output = GarnetObjectStoreOutput.FromPinnedPointer(outputPtr, outputLength); + basicContext.Upsert(key.ReadOnlySpan, valB); if (!output.SpanByteAndMemory.IsSpanByte) output.SpanByteAndMemory.Memory.Dispose(); } - static void ObjectStoreRMW(BasicContext basicContext, + static void ObjectStoreRMW(BasicContext basicContext, ObjectInput objectStoreInput, byte* ptr, byte* outputPtr, int outputLength) { var curr = ptr + sizeof(AofHeader); - ref var key = ref Unsafe.AsRef(curr); + var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); curr += key.TotalSize; - var keyB = key.ToByteArray(); // Reconstructing ObjectInput // input - objectStoreInput.DeserializeFrom(curr); + _ = objectStoreInput.DeserializeFrom(curr); // Call RMW with the reconstructed key & ObjectInput - var output = new GarnetObjectStoreOutput(new(outputPtr, outputLength)); - if (basicContext.RMW(ref keyB, ref objectStoreInput, ref output).IsPending) + var output = GarnetObjectStoreOutput.FromPinnedPointer(outputPtr, outputLength); + if (basicContext.RMW(key.ReadOnlySpan, ref objectStoreInput, ref output).IsPending) basicContext.CompletePending(true); if (!output.SpanByteAndMemory.IsSpanByte) output.SpanByteAndMemory.Memory.Dispose(); } - static void ObjectStoreDelete(BasicContext basicContext, byte* ptr) + static void ObjectStoreDelete(BasicContext basicContext, byte* ptr) { - ref var key = ref Unsafe.AsRef(ptr + sizeof(AofHeader)); - var keyB = key.ToByteArray(); - basicContext.Delete(ref keyB); + 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 3e225dd16a8..2d70f4ff8e3 100644 --- a/libs/server/API/GarnetApi.cs +++ b/libs/server/API/GarnetApi.cs @@ -8,22 +8,20 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; - // Example aliases: - // using BasicGarnetApi = GarnetApi, BasicContext>; - // using LockableGarnetApi = GarnetApi, LockableContext>; + // See TransactionManager.cs for aliases BasicGarnetApi and TransactionalGarnetApi /// /// 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; @@ -38,126 +36,123 @@ internal GarnetApi(StorageSession storageSession, TContext context, TObjectConte #region WATCH /// - public void WATCH(ArgSlice key, StoreType type) - => storageSession.WATCH(key, type); - - /// - public void WATCH(byte[] key, StoreType type) + public void WATCH(PinnedSpanByte key, StoreType type) => storageSession.WATCH(key, type); #endregion #region GET /// - public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.GET(ref key, ref input, ref output, ref context); + public GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.GET(key, ref input, ref output, ref context); /// - public GarnetStatus GET_WithPending(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending) - => storageSession.GET_WithPending(ref key, ref input, ref output, ctx, out pending, ref context); + public GarnetStatus GET_WithPending(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending) + => storageSession.GET_WithPending(key.ReadOnlySpan, ref input, ref output, ctx, out pending, ref context); /// public bool GET_CompletePending((GarnetStatus, SpanByteAndMemory)[] outputArr, bool wait = false) => storageSession.GET_CompletePending(outputArr, wait, ref context); - public bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait) + public bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait) => storageSession.GET_CompletePending(out completedOutputs, wait, ref context); /// - public unsafe GarnetStatus GETForMemoryResult(ArgSlice key, out MemoryResult value) + public unsafe GarnetStatus GETForMemoryResult(PinnedSpanByte key, out MemoryResult value) => storageSession.GET(key, out value, ref context); /// - public unsafe GarnetStatus GET(ArgSlice key, out ArgSlice value) + public unsafe GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte value) => storageSession.GET(key, out value, ref context); /// - public GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput value) + public GarnetStatus GET(PinnedSpanByte key, out GarnetObjectStoreOutput value) => storageSession.GET(key, out value, ref objectContext); /// - public GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) + public GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) => storageSession.LCS(key1, key2, ref output, lenOnly, withIndices, withMatchLen, minMatchLen); #endregion #region GETEX /// - public GarnetStatus GETEX(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.GETEX(ref key, ref input, ref output, ref context); + public GarnetStatus GETEX(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.GETEX(key, ref input, ref output, ref context); #endregion #region GETRANGE /// - public GarnetStatus GETRANGE(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.GETRANGE(ref key, ref input, ref output, ref context); + public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.GETRANGE(key, ref input, ref output, ref context); #endregion #region TTL /// - public GarnetStatus TTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.TTL(ref key, storeType, ref output, ref context, ref objectContext); + public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) + => storageSession.TTL(key, storeType, ref output, ref context, ref objectContext); /// - public GarnetStatus PTTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.TTL(ref key, storeType, ref output, ref context, ref objectContext, milliseconds: true); + 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(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.EXPIRETIME(ref key, storeType, ref output, ref context, ref objectContext); + public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) + => storageSession.EXPIRETIME(key, storeType, ref output, ref context, ref objectContext); /// - public GarnetStatus PEXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) - => storageSession.EXPIRETIME(ref key, storeType, ref output, ref context, ref objectContext, milliseconds: true); + 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(ref SpanByte key, ref SpanByte value) - => storageSession.SET(ref key, ref value, ref context); + public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value) + => storageSession.SET(key, value, ref context); /// - public GarnetStatus SET(ref SpanByte key, ref RawStringInput input, ref SpanByte value) - => storageSession.SET(ref key, ref input, ref value, ref context); + public GarnetStatus SET(PinnedSpanByte key, ref RawStringInput input, PinnedSpanByte value) + => storageSession.SET(key, ref input, value, ref context); /// - public GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input) - => storageSession.SET_Conditional(ref key, ref input, ref context); + public GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input) + => storageSession.SET_Conditional(key, ref input, ref context); /// - public GarnetStatus DEL_Conditional(ref SpanByte key, ref RawStringInput input) - => storageSession.DEL_Conditional(ref key, ref input, ref context); + public GarnetStatus DEL_Conditional(PinnedSpanByte key, ref RawStringInput input) + => storageSession.DEL_Conditional(key, ref input, ref context); /// - public GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.SET_Conditional(ref key, ref input, ref output, ref context); + public GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.SET_Conditional(key, ref input, ref output, ref context); /// - public GarnetStatus SET(ArgSlice key, Memory value) + public GarnetStatus SET(PinnedSpanByte key, Memory value) => storageSession.SET(key, value, ref context); /// - public GarnetStatus SET(ArgSlice key, ArgSlice value) - => storageSession.SET(key, value, ref context); + public GarnetStatus SET(PinnedSpanByte key, IGarnetObject value) + => storageSession.SET(key, value, ref objectContext); /// - public GarnetStatus SET(byte[] key, IGarnetObject value) - => storageSession.SET(key, value, ref objectContext); + public GarnetStatus SET(ref TSourceLogRecord srcLogRecord, StoreType storeType) + where TSourceLogRecord : ISourceLogRecord + => storageSession.SET(ref srcLogRecord, storeType, ref context, ref objectContext); #endregion #region SETEX /// - public unsafe GarnetStatus SETEX(ArgSlice key, ArgSlice value, ArgSlice expiryMs) + public unsafe GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, PinnedSpanByte expiryMs) => storageSession.SETEX(key, value, expiryMs, ref context); /// - public GarnetStatus SETEX(ArgSlice key, ArgSlice value, TimeSpan expiry) + public GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, TimeSpan expiry) => storageSession.SETEX(key, value, expiry, ref context); #endregion @@ -165,7 +160,7 @@ public GarnetStatus SETEX(ArgSlice key, ArgSlice value, TimeSpan expiry) #region SETRANGE /// - public GarnetStatus SETRANGE(ArgSlice key, ref RawStringInput input, ref ArgSlice output) + public GarnetStatus SETRANGE(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output) => storageSession.SETRANGE(key, ref input, ref output, ref context); #endregion @@ -179,105 +174,93 @@ public GarnetStatus MSET_Conditional(ref RawStringInput input) => #region APPEND /// - public GarnetStatus APPEND(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.APPEND(ref key, ref input, ref output, ref context); + public GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.APPEND(key, ref input, ref output, ref context); /// - public GarnetStatus APPEND(ArgSlice key, ArgSlice value, ref ArgSlice output) + public GarnetStatus APPEND(PinnedSpanByte key, PinnedSpanByte value, ref PinnedSpanByte output) => storageSession.APPEND(key, value, ref output, ref context); #endregion #region RENAME /// - public GarnetStatus RENAME(ArgSlice oldKey, ArgSlice newKey, bool withEtag = false, StoreType storeType = StoreType.All) + public GarnetStatus RENAME(PinnedSpanByte oldKey, PinnedSpanByte newKey, bool withEtag = false, StoreType storeType = StoreType.All) => storageSession.RENAME(oldKey, newKey, storeType, withEtag); /// - public GarnetStatus RENAMENX(ArgSlice oldKey, ArgSlice newKey, out int result, bool withEtag = false, StoreType storeType = StoreType.All) + public GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out int result, bool withEtag = false, StoreType storeType = StoreType.All) => storageSession.RENAMENX(oldKey, newKey, storeType, out result, withEtag); #endregion #region EXISTS /// - public GarnetStatus EXISTS(ArgSlice key, StoreType storeType = StoreType.All) + public GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All) => storageSession.EXISTS(key, storeType, ref context, ref objectContext); #endregion #region EXPIRE /// - public unsafe GarnetStatus EXPIRE(ArgSlice key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All) + 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(ArgSlice key, ArgSlice expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) + 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(ArgSlice key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) + 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(ArgSlice key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) + 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(ArgSlice key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None) + 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 PERSIST /// - public unsafe GarnetStatus PERSIST(ArgSlice key, StoreType storeType = StoreType.All) + 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(ArgSlice key, ref RawStringInput input, ref ArgSlice output) + public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output) => storageSession.Increment(key, ref input, ref output, ref context); /// - public GarnetStatus Increment(ArgSlice key, out long output, long incrementCount = 1) + public GarnetStatus Increment(PinnedSpanByte key, out long output, long incrementCount = 1) => storageSession.Increment(key, out output, incrementCount, ref context); /// - public GarnetStatus Decrement(ArgSlice key, out long output, long decrementCount = 1) + public GarnetStatus Decrement(PinnedSpanByte key, out long output, long decrementCount = 1) => Increment(key, out output, -decrementCount); #endregion #region DELETE /// - public GarnetStatus DELETE(ArgSlice key, StoreType storeType = StoreType.All) - => storageSession.DELETE(key, storeType, ref context, ref objectContext); - - /// - public GarnetStatus DELETE(ref SpanByte key, StoreType storeType = StoreType.All) - => storageSession.DELETE(ref key, storeType, ref context, ref objectContext); - - /// - public GarnetStatus DELETE(byte[] key, StoreType storeType = StoreType.All) + public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All) => storageSession.DELETE(key, storeType, ref context, ref objectContext); #endregion #region GETDEL /// - public GarnetStatus GETDEL(ref SpanByte key, ref SpanByteAndMemory output) - => storageSession.GETDEL(ref key, ref output, ref context); - - /// - public GarnetStatus GETDEL(ArgSlice key, ref SpanByteAndMemory output) + public GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output) => storageSession.GETDEL(key, ref output, ref context); #endregion #region TYPE /// - public GarnetStatus GetKeyType(ArgSlice key, out string typeName) + public GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName) => storageSession.GetKeyType(key, out typeName, ref context, ref objectContext); #endregion @@ -285,53 +268,53 @@ public GarnetStatus GetKeyType(ArgSlice key, out string typeName) #region MEMORY /// - public GarnetStatus MemoryUsageForKey(ArgSlice key, out long memoryUsage, int samples = 0) + 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(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.RMW_MainStore(ref key, ref input, ref output, ref context); + public GarnetStatus RMW_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); /// - public GarnetStatus Read_MainStore(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.Read_MainStore(ref key, ref input, ref output, ref context); + public GarnetStatus Read_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); /// - public GarnetStatus RMW_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) - => storageSession.RMW_ObjectStore(ref key, ref input, ref output, ref objectContext); + public GarnetStatus RMW_ObjectStore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + => storageSession.RMW_ObjectStore(key.ReadOnlySpan, ref input, ref output, ref objectContext); /// - public GarnetStatus Read_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) - => storageSession.Read_ObjectStore(ref key, ref input, ref output, ref objectContext); + public GarnetStatus Read_ObjectStore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + => storageSession.Read_ObjectStore(key.ReadOnlySpan, ref input, ref output, ref objectContext); #endregion #region Bitmap Methods /// - public GarnetStatus StringSetBit(ArgSlice key, ArgSlice offset, bool bit, out bool previous) + public GarnetStatus StringSetBit(PinnedSpanByte key, PinnedSpanByte offset, bool bit, out bool previous) => storageSession.StringSetBit(key, offset, bit, out previous, ref context); /// - public GarnetStatus StringSetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.StringSetBit(ref key, ref input, ref output, ref context); + public GarnetStatus StringSetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.StringSetBit(key, ref input, ref output, ref context); /// - public GarnetStatus StringGetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.StringGetBit(ref key, ref input, ref output, ref context); + public GarnetStatus StringGetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.StringGetBit(key, ref input, ref output, ref context); /// - public GarnetStatus StringGetBit(ArgSlice key, ArgSlice offset, out bool bValue) + public GarnetStatus StringGetBit(PinnedSpanByte key, PinnedSpanByte offset, out bool bValue) => storageSession.StringGetBit(key, offset, out bValue, ref context); /// - public GarnetStatus StringBitCount(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.StringBitCount(ref key, ref input, ref output, ref context); + public GarnetStatus StringBitCount(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.StringBitCount(key, ref input, ref output, ref context); /// - public GarnetStatus StringBitCount(ArgSlice key, long start, long end, out long result, bool useBitInterval = false) + public GarnetStatus StringBitCount(PinnedSpanByte key, long start, long end, out long result, bool useBitInterval = false) => storageSession.StringBitCount(key, start, end, useBitInterval, out result, ref context); /// @@ -339,34 +322,34 @@ public GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOperation => storageSession.StringBitOperation(ref input, bitOp, out result); /// - public GarnetStatus StringBitOperation(BitmapOperation bitop, ArgSlice destinationKey, ArgSlice[] keys, out long result) + public GarnetStatus StringBitOperation(BitmapOperation bitop, PinnedSpanByte destinationKey, PinnedSpanByte[] keys, out long result) => storageSession.StringBitOperation(bitop, destinationKey, keys, out result); /// - public GarnetStatus StringBitPosition(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.StringBitPosition(ref key, ref input, ref output, ref context); + public GarnetStatus StringBitPosition(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.StringBitPosition(key, ref input, ref output, ref context); /// - public GarnetStatus StringBitField(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) - => storageSession.StringBitField(ref key, ref input, secondaryCommand, ref output, ref context); + public GarnetStatus StringBitField(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) + => storageSession.StringBitField(key, ref input, secondaryCommand, ref output, ref context); /// - public GarnetStatus StringBitFieldReadOnly(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) - => storageSession.StringBitFieldReadOnly(ref key, ref input, secondaryCommand, ref output, ref context); + public GarnetStatus StringBitFieldReadOnly(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) + => storageSession.StringBitFieldReadOnly(key, ref input, secondaryCommand, ref output, ref context); /// - public GarnetStatus StringBitField(ArgSlice key, List commandArguments, out List result) + public GarnetStatus StringBitField(PinnedSpanByte key, List commandArguments, out List result) => storageSession.StringBitField(key, commandArguments, out result, ref context); #endregion #region HyperLogLog Methods /// - public GarnetStatus HyperLogLogAdd(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) - => storageSession.HyperLogLogAdd(ref key, ref input, ref output, ref context); + public GarnetStatus HyperLogLogAdd(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + => storageSession.HyperLogLogAdd(key, ref input, ref output, ref context); /// - public GarnetStatus HyperLogLogAdd(ArgSlice key, string[] elements, out bool updated) + public GarnetStatus HyperLogLogAdd(PinnedSpanByte key, string[] elements, out bool updated) => storageSession.HyperLogLogAdd(key, elements, out updated, ref context); /// @@ -374,7 +357,7 @@ public GarnetStatus HyperLogLogLength(ref RawStringInput input, out long count, => storageSession.HyperLogLogLength(ref input, out count, out error, ref context); /// - public GarnetStatus HyperLogLogLength(Span keys, out long count) + public GarnetStatus HyperLogLogLength(Span keys, out long count) => storageSession.HyperLogLogLength(keys, out count, ref context); /// @@ -385,7 +368,7 @@ public GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool error) #region Server Methods /// - public List GetDbKeys(ArgSlice pattern) + public List GetDbKeys(PinnedSpanByte pattern) => storageSession.DBKeys(pattern); /// @@ -393,25 +376,25 @@ public int GetDbSize() => storageSession.DbSize(); /// - public bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out long storeCursor, out List Keys, long count = 10, ReadOnlySpan type = default) + public readonly bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, out long storeCursor, out List Keys, long count = 10, ReadOnlySpan type = default) => storageSession.DbScan(patternB, allKeys, cursor, out storeCursor, out Keys, count, type); /// - public bool IterateMainStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + public readonly bool IterateMainStore(ref TScanFunctions scanFunctions, long untilAddress = -1) + where TScanFunctions : IScanIteratorFunctions => storageSession.IterateMainStore(ref scanFunctions, untilAddress); /// - public ITsavoriteScanIterator IterateMainStore() + public readonly ITsavoriteScanIterator IterateMainStore() => storageSession.IterateMainStore(); /// - public bool IterateObjectStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + public readonly bool IterateObjectStore(ref TScanFunctions scanFunctions, long untilAddress = -1) + where TScanFunctions : IScanIteratorFunctions => storageSession.IterateObjectStore(ref scanFunctions, untilAddress); /// - public ITsavoriteScanIterator IterateObjectStore() + public readonly ITsavoriteScanIterator IterateObjectStore() => storageSession.IterateObjectStore(); #endregion @@ -419,8 +402,8 @@ public ITsavoriteScanIterator IterateObjectStore() #region Common Methods /// - public GarnetStatus ObjectScan(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) - => storageSession.ObjectScan(key, ref input, ref output, ref objectContext); + public GarnetStatus ObjectScan(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + => storageSession.ObjectScan(key.ReadOnlySpan, ref input, ref output, ref objectContext); /// public int GetScratchBufferOffset() diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs index e5c579f49c6..dea5b54afce 100644 --- a/libs/server/API/GarnetApiObjectCommands.cs +++ b/libs/server/API/GarnetApiObjectCommands.cs @@ -7,194 +7,194 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = 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 /// - public GarnetStatus SortedSetAdd(ArgSlice key, ArgSlice score, ArgSlice member, out int zaddCount) + public GarnetStatus SortedSetAdd(PinnedSpanByte key, PinnedSpanByte score, PinnedSpanByte member, out int zaddCount) => storageSession.SortedSetAdd(key, score, member, out zaddCount, ref objectContext); /// - public GarnetStatus SortedSetAdd(ArgSlice key, (ArgSlice score, ArgSlice member)[] inputs, out int zaddCount) + public GarnetStatus SortedSetAdd(PinnedSpanByte key, (PinnedSpanByte score, PinnedSpanByte member)[] inputs, out int zaddCount) => storageSession.SortedSetAdd(key, inputs, out zaddCount, ref objectContext); /// - public GarnetStatus SortedSetAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetAdd(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetRangeStore(ArgSlice dstKey, ArgSlice srcKey, ref ObjectInput input, out int result) + public GarnetStatus SortedSetRangeStore(PinnedSpanByte dstKey, PinnedSpanByte srcKey, ref ObjectInput input, out int result) => storageSession.SortedSetRangeStore(dstKey, srcKey, ref input, out result, ref objectContext); /// - public GarnetStatus SortedSetRemove(ArgSlice key, ArgSlice member, out int zremCount) - => storageSession.SortedSetRemove(key.ToArray(), member, out zremCount, ref objectContext); + public GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte member, out int zremCount) + => storageSession.SortedSetRemove(key, member, out zremCount, ref objectContext); /// - public GarnetStatus SortedSetRemove(ArgSlice key, ArgSlice[] members, out int zaddCount) - => storageSession.SortedSetRemove(key.ToArray(), members, out zaddCount, ref objectContext); + public GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int zaddCount) + => storageSession.SortedSetRemove(key, members, out zaddCount, ref objectContext); /// - public GarnetStatus SortedSetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SortedSetRemove(key, ref input, out output, ref objectContext); /// - public GarnetStatus SortedSetLength(ArgSlice key, out int len) + public GarnetStatus SortedSetLength(PinnedSpanByte key, out int len) => storageSession.SortedSetLength(key, out len, ref objectContext); /// - public GarnetStatus SortedSetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SortedSetLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus SortedSetRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetRange(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetScore(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetScore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetScore(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetScores(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetScores(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetScores(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetPop(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out ArgSlice poppedKey, out (ArgSlice member, ArgSlice score)[] pairs) + public GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out PinnedSpanByte poppedKey, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs) => storageSession.SortedSetMPop(keys, count, lowScoresFirst, out poppedKey, out pairs); /// - public GarnetStatus SortedSetPop(ArgSlice key, out (ArgSlice member, ArgSlice score)[] pairs, int count = 1, bool lowScoresFirst = true) + public GarnetStatus SortedSetPop(PinnedSpanByte key, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs, int count = 1, bool lowScoresFirst = true) => storageSession.SortedSetPop(key, count, lowScoresFirst, out pairs, ref objectContext); /// - public GarnetStatus SortedSetCount(ArgSlice key, ArgSlice minScore, ArgSlice maxScore, out int numElements) + public GarnetStatus SortedSetCount(PinnedSpanByte key, PinnedSpanByte minScore, PinnedSpanByte maxScore, out int numElements) => storageSession.SortedSetCount(key, minScore, maxScore, out numElements, ref objectContext); /// - public GarnetStatus SortedSetCount(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetCount(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetLengthByValue(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SortedSetLengthByValue(key, ref input, out output, ref objectContext); /// - public GarnetStatus SortedSetRemoveRangeByLex(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SortedSetRemoveRangeByLex(key, ref input, out output, ref objectContext); /// - public GarnetStatus SortedSetRemoveRangeByLex(ArgSlice key, string min, string max, out int countRemoved) + public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, string min, string max, out int countRemoved) => storageSession.SortedSetRemoveRangeByLex(key, min, max, out countRemoved, ref objectContext); /// - public GarnetStatus SortedSetRemoveRangeByScore(ArgSlice key, string min, string max, out int countRemoved) + public GarnetStatus SortedSetRemoveRangeByScore(PinnedSpanByte key, string min, string max, out int countRemoved) => storageSession.SortedSetRemoveRangeByScore(key, min, max, out countRemoved, ref objectContext); /// - public GarnetStatus SortedSetRemoveRangeByRank(ArgSlice key, int start, int stop, out int countRemoved) + public GarnetStatus SortedSetRemoveRangeByRank(PinnedSpanByte key, int start, int stop, out int countRemoved) => storageSession.SortedSetRemoveRangeByRank(key, start, stop, out countRemoved, ref objectContext); /// - public GarnetStatus SortedSetIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetIncrement(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetIncrement(ArgSlice key, double increment, ArgSlice member, out double newScore) + public GarnetStatus SortedSetIncrement(PinnedSpanByte key, double increment, PinnedSpanByte member, out double newScore) => storageSession.SortedSetIncrement(key, increment, member, out newScore, ref objectContext); /// - public GarnetStatus SortedSetRemoveRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRemoveRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetRemoveRange(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetRank(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRank(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetRank(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetRank(ArgSlice key, ArgSlice member, bool reverse, out long? rank) + public GarnetStatus SortedSetRank(PinnedSpanByte key, PinnedSpanByte member, bool reverse, out long? rank) => storageSession.SortedSetRank(key, member, reverse, out rank, ref objectContext); /// - public GarnetStatus SortedSetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetRandomMember(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetRange(ArgSlice key, ArgSlice min, ArgSlice max, SortedSetOrderOperation sortedSetOrderOperation, out ArgSlice[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) + public GarnetStatus SortedSetRange(PinnedSpanByte key, PinnedSpanByte min, PinnedSpanByte max, SortedSetOrderOperation sortedSetOrderOperation, out PinnedSpanByte[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) => storageSession.SortedSetRange(key, min, max, sortedSetOrderOperation, ref objectContext, out elements, out error, withScores, reverse, limit); /// - public GarnetStatus SortedSetDifference(ArgSlice[] keys, out Dictionary pairs) + public GarnetStatus SortedSetDifference(PinnedSpanByte[] keys, out Dictionary pairs) => storageSession.SortedSetDifference(keys, out pairs); /// - public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) => storageSession.SortedSetUnion(keys, weights, aggregateType, out pairs); /// - public GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySpan keys, out int count) + public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, out int count) => storageSession.SortedSetDifferenceStore(destinationKey, keys, out count); - public GarnetStatus SortedSetUnionStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) + public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) => storageSession.SortedSetUnionStore(destinationKey, keys, weights, aggregateType, out count); /// - public GarnetStatus SortedSetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus SortedSetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) => storageSession.ObjectScan(GarnetObjectType.SortedSet, key, cursor, match, count, out items, ref objectContext); /// - public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) => storageSession.SortedSetIntersect(keys, weights, aggregateType, out pairs); /// - public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) => storageSession.SortedSetIntersectLength(keys, limit, out count); /// - public GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) + public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) => storageSession.SortedSetIntersectStore(destinationKey, keys, weights, aggregateType, out count); /// - public GarnetStatus SortedSetExpire(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetExpire(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetExpire(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetExpire(ArgSlice key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results) + public GarnetStatus SortedSetExpire(PinnedSpanByte key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results) => storageSession.SortedSetExpire(key, members, expireAt, expireOption, out results, ref objectContext); /// - public GarnetStatus SortedSetPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetPersist(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetPersist(ArgSlice key, ReadOnlySpan members, out int[] results) + public GarnetStatus SortedSetPersist(PinnedSpanByte key, ReadOnlySpan members, out int[] results) => storageSession.SortedSetPersist(key, members, out results, ref objectContext); /// - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SortedSetTimeToLive(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySpan members, out TimeSpan[] expireIn) + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ReadOnlySpan members, out TimeSpan[] expireIn) => storageSession.SortedSetTimeToLive(key, members, out expireIn, ref objectContext); /// - public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input) + public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input) => storageSession.SortedSetCollect(keys, ref input, ref objectContext); /// @@ -202,7 +202,7 @@ public GarnetStatus SortedSetCollect() => storageSession.SortedSetCollect(ref objectContext); /// - public GarnetStatus SortedSetCollect(ReadOnlySpan keys) + public GarnetStatus SortedSetCollect(ReadOnlySpan keys) => storageSession.SortedSetCollect(keys, ref objectContext); #endregion @@ -210,20 +210,20 @@ public GarnetStatus SortedSetCollect(ReadOnlySpan keys) #region Geospatial commands /// - public GarnetStatus GeoAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus GeoAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.GeoAdd(key, ref input, ref output, ref objectContext); /// - public GarnetStatus GeoCommands(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus GeoCommands(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.GeoCommands(key, ref input, ref output, ref objectContext); /// - public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, + public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output) => storageSession.GeoSearchReadOnly(key, ref opts, ref input, ref output, ref objectContext); /// - public GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearchOptions opts, + public GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output) => storageSession.GeoSearchStore(key, destinationKey, ref opts, ref input, ref output, ref objectContext); #endregion @@ -233,105 +233,105 @@ public GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref Ge #region PUSHPOP /// - public GarnetStatus ListRightPush(ArgSlice key, ArgSlice element, out int itemsCount, bool whenExists = false) + public GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte element, out int itemsCount, bool whenExists = false) => storageSession.ListPush(key, element, whenExists ? ListOperation.RPUSHX : ListOperation.RPUSH, out itemsCount, ref objectContext); /// - public GarnetStatus ListRightPush(ArgSlice key, ArgSlice[] elements, out int itemsCount, bool whenExists = false) + public GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int itemsCount, bool whenExists = false) => storageSession.ListPush(key, elements, whenExists ? ListOperation.RPUSHX : ListOperation.RPUSH, out itemsCount, ref objectContext); /// - public GarnetStatus ListRightPush(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.ListPush(key, ref input, out output, ref objectContext); /// - public GarnetStatus ListLeftPush(ArgSlice key, ArgSlice[] elements, out int itemsCount, bool onlyWhenExists = false) + public GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int itemsCount, bool onlyWhenExists = false) => storageSession.ListPush(key, elements, onlyWhenExists ? ListOperation.LPUSHX : ListOperation.LPUSH, out itemsCount, ref objectContext); /// - public GarnetStatus ListLeftPush(ArgSlice key, ArgSlice element, out int count, bool onlyWhenExists = false) + public GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte element, out int count, bool onlyWhenExists = false) => storageSession.ListPush(key, element, onlyWhenExists ? ListOperation.LPUSHX : ListOperation.LPUSH, out count, ref objectContext); /// - public GarnetStatus ListLeftPush(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.ListPush(key, ref input, out output, ref objectContext); /// - public GarnetStatus ListPosition(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListPosition(key, ref input, ref output, ref objectContext); /// - public GarnetStatus ListLeftPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListLeftPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListPop(key, ref input, ref output, ref objectContext); /// - public unsafe GarnetStatus ListLeftPop(ArgSlice key, out ArgSlice element) + public unsafe GarnetStatus ListLeftPop(PinnedSpanByte key, out PinnedSpanByte element) => storageSession.ListPop(key, ListOperation.LPOP, ref objectContext, out element); /// - public GarnetStatus ListLeftPop(ArgSlice key, int count, out ArgSlice[] poppedElements) + public GarnetStatus ListLeftPop(PinnedSpanByte key, int count, out PinnedSpanByte[] poppedElements) => storageSession.ListPop(key, count, ListOperation.LPOP, ref objectContext, out poppedElements); /// - public GarnetStatus ListLeftPop(ArgSlice[] keys, int count, out ArgSlice poppedKey, out ArgSlice[] poppedElements) + public GarnetStatus ListLeftPop(PinnedSpanByte[] keys, int count, out PinnedSpanByte poppedKey, out PinnedSpanByte[] poppedElements) => storageSession.ListPopMultiple(keys, OperationDirection.Left, count, ref objectContext, out poppedKey, out poppedElements); /// - public GarnetStatus ListRightPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListRightPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListPop(key, ref input, ref output, ref objectContext); /// - public unsafe GarnetStatus ListRightPop(ArgSlice key, out ArgSlice element) + public unsafe GarnetStatus ListRightPop(PinnedSpanByte key, out PinnedSpanByte element) => storageSession.ListPop(key, ListOperation.RPOP, ref objectContext, out element); /// - public GarnetStatus ListRightPop(ArgSlice key, int count, out ArgSlice[] poppedElements) + public GarnetStatus ListRightPop(PinnedSpanByte key, int count, out PinnedSpanByte[] poppedElements) => storageSession.ListPop(key, count, ListOperation.RPOP, ref objectContext, out poppedElements); /// - public GarnetStatus ListRightPop(ArgSlice[] keys, int count, out ArgSlice poppedKey, out ArgSlice[] poppedElements) + public GarnetStatus ListRightPop(PinnedSpanByte[] keys, int count, out PinnedSpanByte poppedKey, out PinnedSpanByte[] poppedElements) => storageSession.ListPopMultiple(keys, OperationDirection.Right, count, ref objectContext, out poppedKey, out poppedElements); #endregion /// - public GarnetStatus ListLength(ArgSlice key, out int count) + public GarnetStatus ListLength(PinnedSpanByte key, out int count) => storageSession.ListLength(key, ref objectContext, out count); /// - public GarnetStatus ListLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.ListLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus ListMove(ArgSlice source, ArgSlice destination, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element) + public GarnetStatus ListMove(PinnedSpanByte source, PinnedSpanByte destination, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element) => storageSession.ListMove(source, destination, sourceDirection, destinationDirection, out element); /// - public bool ListTrim(ArgSlice key, int start, int stop) + public bool ListTrim(PinnedSpanByte key, int start, int stop) => storageSession.ListTrim(key, start, stop, ref objectContext); /// - public GarnetStatus ListTrim(byte[] key, ref ObjectInput input) + public GarnetStatus ListTrim(PinnedSpanByte key, ref ObjectInput input) => storageSession.ListTrim(key, ref input, ref objectContext); /// - public GarnetStatus ListRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListRange(key, ref input, ref output, ref objectContext); /// - public GarnetStatus ListInsert(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.ListInsert(key, ref input, out output, ref objectContext); /// - public GarnetStatus ListIndex(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListIndex(key, ref input, ref output, ref objectContext); /// - public GarnetStatus ListRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.ListRemove(key, ref input, out output, ref objectContext); /// - public GarnetStatus ListSet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListSet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.ListSet(key, ref input, ref output, ref objectContext); #endregion @@ -339,102 +339,102 @@ public GarnetStatus ListSet(byte[] key, ref ObjectInput input, ref GarnetObjectS #region Set Methods /// - public GarnetStatus SetAdd(ArgSlice key, ArgSlice member, out int saddCount) + public GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte member, out int saddCount) => storageSession.SetAdd(key, member, out saddCount, ref objectContext); /// - public GarnetStatus SetAdd(ArgSlice key, ArgSlice[] members, out int saddCount) + public GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int saddCount) => storageSession.SetAdd(key, members, out saddCount, ref objectContext); /// - public GarnetStatus SetAdd(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SetAdd(key, ref input, out output, ref objectContext); /// - public GarnetStatus SetRemove(ArgSlice key, ArgSlice member, out int sremCount) + public GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte member, out int sremCount) => storageSession.SetRemove(key, member, out sremCount, ref objectContext); /// - public GarnetStatus SetRemove(ArgSlice key, ArgSlice[] members, out int sremCount) + public GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int sremCount) => storageSession.SetRemove(key, members, out sremCount, ref objectContext); /// - public GarnetStatus SetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SetRemove(key, ref input, out output, ref objectContext); /// - public GarnetStatus SetLength(ArgSlice key, out int count) + public GarnetStatus SetLength(PinnedSpanByte key, out int count) => storageSession.SetLength(key, out count, ref objectContext); /// - public GarnetStatus SetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.SetLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus SetMembers(ArgSlice key, out ArgSlice[] members) + public GarnetStatus SetMembers(PinnedSpanByte key, out PinnedSpanByte[] members) => storageSession.SetMembers(key, out members, ref objectContext); /// - public GarnetStatus SetMembers(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SetMembers(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SetIsMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SetIsMember(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] members, out int[] result) + public GarnetStatus SetIsMember(PinnedSpanByte key, PinnedSpanByte[] members, out int[] result) => storageSession.SetIsMember(key, members, out result, ref objectContext); /// - public GarnetStatus SetPop(ArgSlice key, out ArgSlice member) + public GarnetStatus SetPop(PinnedSpanByte key, out PinnedSpanByte member) => storageSession.SetPop(key, out member, ref objectContext); /// - public GarnetStatus SetPop(ArgSlice key, int count, out ArgSlice[] members) + public GarnetStatus SetPop(PinnedSpanByte key, int count, out PinnedSpanByte[] members) => storageSession.SetPop(key, count, out members, ref objectContext); /// - public GarnetStatus SetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SetPop(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.SetRandomMember(key, ref input, ref output, ref objectContext); /// - public GarnetStatus SetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus SetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) => storageSession.ObjectScan(GarnetObjectType.Set, key, cursor, match, count, out items, ref objectContext); /// - public GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey, ArgSlice member, out int smoveResult) + public GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, PinnedSpanByte member, out int smoveResult) => storageSession.SetMove(sourceKey, destinationKey, member, out smoveResult); - public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output) => storageSession.SetUnion(keys, out output); /// - public GarnetStatus SetUnionStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) => storageSession.SetUnionStore(key, keys, out count); /// - public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet members) + public GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet members) => storageSession.SetDiff(keys, out members); /// - public GarnetStatus SetDiffStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) => storageSession.SetDiffStore(key, keys, out count); /// - public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet output) => storageSession.SetIntersect(keys, out output); /// - public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) => storageSession.SetIntersectLength(keys, limit, out count); /// - public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) => storageSession.SetIntersectStore(key, keys, out count); #endregion @@ -442,126 +442,125 @@ public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count #region Hash Methods /// - public GarnetStatus HashSet(ArgSlice key, ArgSlice field, ArgSlice value, out int count) + public GarnetStatus HashSet(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int count) => storageSession.HashSet(key, field, value, out count, ref objectContext); /// - public GarnetStatus HashSetWhenNotExists(ArgSlice key, ArgSlice field, ArgSlice value, out int count) + public GarnetStatus HashSetWhenNotExists(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int count) => storageSession.HashSet(key, field, value, out count, ref objectContext, nx: true); /// - public GarnetStatus HashSet(ArgSlice key, (ArgSlice field, ArgSlice value)[] elements, out int count) + public GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpanByte value)[] elements, out int count) => storageSession.HashSet(key, elements, out count, ref objectContext); /// - public GarnetStatus HashSet(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.HashSet(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashDelete(ArgSlice key, ArgSlice field, out int count) + public GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte field, out int count) => storageSession.HashDelete(key, field, out count, ref objectContext); /// - public GarnetStatus HashDelete(ArgSlice key, ArgSlice[] fields, out int count) + public GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte[] fields, out int count) => storageSession.HashDelete(key, fields, out count, ref objectContext); /// - public GarnetStatus HashGet(ArgSlice key, ArgSlice field, out ArgSlice value) + public GarnetStatus HashGet(PinnedSpanByte key, PinnedSpanByte field, out PinnedSpanByte value) => storageSession.HashGet(key, field, out value, ref objectContext); /// - public GarnetStatus HashGetAll(ArgSlice key, out ArgSlice[] values) + public GarnetStatus HashGetAll(PinnedSpanByte key, out PinnedSpanByte[] values) => storageSession.HashGetAll(key, out values, ref objectContext); /// - public GarnetStatus HashGet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashGet(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashGetAll(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashGetAll(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashGetMultiple(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashGetMultiple(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashGetMultiple(ArgSlice key, ArgSlice[] fields, out ArgSlice[] values) + public GarnetStatus HashGetMultiple(PinnedSpanByte key, PinnedSpanByte[] fields, out PinnedSpanByte[] values) => storageSession.HashGetMultiple(key, fields, out values, ref objectContext); /// - public GarnetStatus HashLength(ArgSlice key, out int count) + public GarnetStatus HashLength(PinnedSpanByte key, out int count) => storageSession.HashLength(key, out count, ref objectContext); /// - public GarnetStatus HashLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.HashLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashStrLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.HashStrLength(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashExists(ArgSlice key, ArgSlice field, out bool exists) + public GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out bool exists) => storageSession.HashExists(key, field, out exists, ref objectContext); /// - public GarnetStatus HashExists(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.HashExists(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashRandomField(ArgSlice key, out ArgSlice field) + public GarnetStatus HashRandomField(PinnedSpanByte key, out PinnedSpanByte field) => storageSession.HashRandomField(key, out field, ref objectContext); /// - public GarnetStatus HashRandomField(ArgSlice key, int count, bool withValues, out ArgSlice[] fields) + public GarnetStatus HashRandomField(PinnedSpanByte key, int count, bool withValues, out PinnedSpanByte[] fields) => storageSession.HashRandomField(key, count, withValues, out fields, ref objectContext); /// - public GarnetStatus HashRandomField(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashRandomField(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashDelete(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) => storageSession.HashDelete(key, ref input, out output, ref objectContext); /// - public GarnetStatus HashKeys(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashKeys(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashVals(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashVals(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashIncrement(byte[] key, ArgSlice input, out ObjectOutputHeader output) + public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output) => storageSession.HashIncrement(key, input, out output, ref objectContext); /// - public GarnetStatus HashIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashIncrement(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashExpire(ArgSlice key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashExpire(PinnedSpanByte key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashExpire(key, expireAt, isMilliseconds, expireOption, ref input, ref output, ref objectContext); /// - public GarnetStatus HashPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashPersist(key, ref input, ref output, ref objectContext); /// - public GarnetStatus HashScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus HashScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) => storageSession.ObjectScan(GarnetObjectType.Hash, key, cursor, match, count, out items, ref objectContext); /// - public GarnetStatus HashTimeToLive(ArgSlice key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output) => storageSession.HashTimeToLive(key, isMilliseconds, isTimestamp, ref input, ref output, ref objectContext); /// - public GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input) + public GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input) => storageSession.HashCollect(keys, ref input, ref objectContext); #endregion } - } \ No newline at end of file diff --git a/libs/server/API/GarnetWatchApi.cs b/libs/server/API/GarnetWatchApi.cs index cff3bf89b81..a3fa1b464b9 100644 --- a/libs/server/API/GarnetWatchApi.cs +++ b/libs/server/API/GarnetWatchApi.cs @@ -23,35 +23,35 @@ public GarnetWatchApi(TGarnetApi garnetApi) #region GET /// - public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + public GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.GET(ref key, ref input, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.GET(key, ref input, ref output); } /// - public GarnetStatus GETForMemoryResult(ArgSlice key, out MemoryResult value) + public GarnetStatus GETForMemoryResult(PinnedSpanByte key, out MemoryResult value) { garnetApi.WATCH(key, StoreType.Main); return garnetApi.GETForMemoryResult(key, out value); } /// - public GarnetStatus GET(ArgSlice key, out ArgSlice value) + public GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte value) { garnetApi.WATCH(key, StoreType.Main); return garnetApi.GET(key, out value); } /// - public GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput value) + public GarnetStatus GET(PinnedSpanByte key, out GarnetObjectStoreOutput value) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.GET(key, out value); } /// - public GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) + public GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) { garnetApi.WATCH(key1, StoreType.Object); garnetApi.WATCH(key2, StoreType.Object); @@ -61,26 +61,26 @@ public GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory outp #region GETRANGE /// - public GarnetStatus GETRANGE(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.GETRANGE(ref key, ref input, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.GETRANGE(key, ref input, ref output); } #endregion #region TTL /// - public GarnetStatus TTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), storeType); - return garnetApi.TTL(ref key, storeType, ref output); + garnetApi.WATCH(key, storeType); + return garnetApi.TTL(key, storeType, ref output); } /// - public GarnetStatus PTTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), storeType); - return garnetApi.PTTL(ref key, storeType, ref output); + garnetApi.WATCH(key, storeType); + return garnetApi.PTTL(key, storeType, ref output); } #endregion @@ -88,17 +88,17 @@ public GarnetStatus PTTL(ref SpanByte key, StoreType storeType, ref SpanByteAndM #region EXPIRETIME /// - public GarnetStatus EXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), storeType); - return garnetApi.EXPIRETIME(ref key, storeType, ref output); + garnetApi.WATCH(key, storeType); + return garnetApi.EXPIRETIME(key, storeType, ref output); } /// - public GarnetStatus PEXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output) + public GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), storeType); - return garnetApi.PEXPIRETIME(ref key, storeType, ref output); + garnetApi.WATCH(key, storeType); + return garnetApi.PEXPIRETIME(key, storeType, ref output); } #endregion @@ -106,91 +106,91 @@ public GarnetStatus PEXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanB #region SortedSet Methods /// - public GarnetStatus SortedSetLength(ArgSlice key, out int zcardCount) + public GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetLength(key, out zcardCount); } /// - public GarnetStatus SortedSetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetLength(key, ref input, out output); } /// - public GarnetStatus SortedSetCount(ArgSlice key, ArgSlice minScore, ArgSlice maxScore, out int numElements) + public GarnetStatus SortedSetCount(PinnedSpanByte key, PinnedSpanByte minScore, PinnedSpanByte maxScore, out int numElements) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetCount(key, minScore, maxScore, out numElements); } /// - public GarnetStatus SortedSetCount(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetCount(key, ref input, ref output); } /// - public GarnetStatus SortedSetLengthByValue(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetLengthByValue(key, ref input, out output); } /// - public GarnetStatus SortedSetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetRandomMember(key, ref input, ref output); } /// - public GarnetStatus SortedSetRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetRange(key, ref input, ref output); } /// - public GarnetStatus SortedSetScore(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetScore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetScore(key, ref input, ref output); } /// - public GarnetStatus SortedSetScores(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetScores(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetScores(key, ref input, ref output); } /// - public GarnetStatus SortedSetRank(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetRank(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetRank(key, ref input, ref output); } /// - public GarnetStatus SortedSetRank(ArgSlice key, ArgSlice member, bool reverse, out long? rank) + public GarnetStatus SortedSetRank(PinnedSpanByte key, PinnedSpanByte member, bool reverse, out long? rank) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetRank(key, member, reverse, out rank); } /// - public GarnetStatus SortedSetRange(ArgSlice key, ArgSlice min, ArgSlice max, SortedSetOrderOperation sortedSetOrderOperation, out ArgSlice[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) + public GarnetStatus SortedSetRange(PinnedSpanByte key, PinnedSpanByte min, PinnedSpanByte max, SortedSetOrderOperation sortedSetOrderOperation, out PinnedSpanByte[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetRange(key, min, max, sortedSetOrderOperation, out elements, out error, withScores, reverse, limit); } /// - public GarnetStatus SortedSetDifference(ArgSlice[] keys, out Dictionary pairs) + public GarnetStatus SortedSetDifference(PinnedSpanByte[] keys, out Dictionary pairs) { foreach (var key in keys) { @@ -200,7 +200,7 @@ public GarnetStatus SortedSetDifference(ArgSlice[] keys, out Dictionary - public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) { foreach (var key in keys) { @@ -210,14 +210,14 @@ public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights } /// - public GarnetStatus GeoCommands(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus GeoCommands(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.GeoCommands(key, ref input, ref output); } /// - public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, + public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output) { garnetApi.WATCH(key, StoreType.Object); @@ -225,14 +225,14 @@ public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, } /// - public GarnetStatus SortedSetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus SortedSetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetScan(key, cursor, match, count, out items); } /// - public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) { foreach (var key in keys) { @@ -242,7 +242,7 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] wei } /// - public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) { foreach (var key in keys) { @@ -252,14 +252,14 @@ public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? l } /// - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetTimeToLive(key, ref input, ref output); } /// - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySpan members, out TimeSpan[] expireIn) + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ReadOnlySpan members, out TimeSpan[] expireIn) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SortedSetTimeToLive(key, members, out expireIn); @@ -270,28 +270,28 @@ public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySpan mem #region List Methods /// - public GarnetStatus ListLength(ArgSlice key, out int count) + public GarnetStatus ListLength(PinnedSpanByte key, out int count) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.ListLength(key, out count); } /// - public GarnetStatus ListLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.ListLength(key, ref input, out output); } /// - public GarnetStatus ListRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.ListRange(key, ref input, ref output); } /// - public GarnetStatus ListIndex(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.ListIndex(key, ref input, ref output); @@ -302,56 +302,56 @@ public GarnetStatus ListIndex(byte[] key, ref ObjectInput input, ref GarnetObjec #region Set Methods /// - public GarnetStatus SetLength(ArgSlice key, out int scardCount) + public GarnetStatus SetLength(PinnedSpanByte key, out int scardCount) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetLength(key, out scardCount); } /// - public GarnetStatus SetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetLength(key, ref input, out output); } /// - public GarnetStatus SetMembers(ArgSlice key, out ArgSlice[] members) + public GarnetStatus SetMembers(PinnedSpanByte key, out PinnedSpanByte[] members) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetMembers(key, out members); } /// - public GarnetStatus SetIsMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetIsMember(key, ref input, ref output); } /// - public GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] members, out int[] result) + public GarnetStatus SetIsMember(PinnedSpanByte key, PinnedSpanByte[] members, out int[] result) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetIsMember(key, members, out result); } /// - public GarnetStatus SetMembers(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetMembers(key, ref input, ref output); } /// - public GarnetStatus SetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus SetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.SetScan(key, cursor, match, count, out items); } /// - public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output) { foreach (var key in keys) { @@ -361,7 +361,7 @@ public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) } /// - public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet output) { foreach (var key in keys) { @@ -371,7 +371,7 @@ public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) } /// - public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet output) { foreach (var key in keys) { @@ -380,7 +380,7 @@ public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet output) return garnetApi.SetDiff(keys, out output); } - public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) { foreach (var key in keys) { @@ -393,124 +393,124 @@ public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, #region Hash Methods /// - public GarnetStatus HashGet(ArgSlice key, ArgSlice field, out ArgSlice value) + public GarnetStatus HashGet(PinnedSpanByte key, PinnedSpanByte field, out PinnedSpanByte value) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGet(key, field, out value); } /// - public GarnetStatus HashGetMultiple(ArgSlice key, ArgSlice[] fields, out ArgSlice[] values) + public GarnetStatus HashGetMultiple(PinnedSpanByte key, PinnedSpanByte[] fields, out PinnedSpanByte[] values) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGetMultiple(key, fields, out values); } /// - public GarnetStatus HashGetAll(ArgSlice key, out ArgSlice[] values) + public GarnetStatus HashGetAll(PinnedSpanByte key, out PinnedSpanByte[] values) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGetAll(key, out values); } /// - public GarnetStatus HashLength(ArgSlice key, out int count) + public GarnetStatus HashLength(PinnedSpanByte key, out int count) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashLength(key, out count); } /// - public GarnetStatus HashExists(ArgSlice key, ArgSlice field, out bool exists) + public GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out bool exists) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashExists(key, field, out exists); } /// - public GarnetStatus HashRandomField(ArgSlice key, int count, bool withValues, out ArgSlice[] fields) + public GarnetStatus HashRandomField(PinnedSpanByte key, int count, bool withValues, out PinnedSpanByte[] fields) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashRandomField(key, count, withValues, out fields); } /// - public GarnetStatus HashRandomField(ArgSlice key, out ArgSlice field) + public GarnetStatus HashRandomField(PinnedSpanByte key, out PinnedSpanByte field) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashRandomField(key, out field); } /// - public GarnetStatus HashRandomField(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashRandomField(key, ref input, ref output); } /// - public GarnetStatus HashGet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGet(key, ref input, ref output); } - public GarnetStatus HashGetAll(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGetAll(key, ref input, ref output); } - public GarnetStatus HashGetMultiple(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashGetMultiple(key, ref input, ref output); } /// - public GarnetStatus HashStrLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashStrLength(key, ref input, out output); } /// - public GarnetStatus HashExists(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashExists(key, ref input, out output); } /// - public GarnetStatus HashKeys(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashKeys(key, ref input, ref output); } /// - public GarnetStatus HashVals(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashVals(key, ref input, ref output); } /// - public GarnetStatus HashLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output) + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashLength(key, ref input, out output); } /// - public GarnetStatus HashScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items) + public GarnetStatus HashScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashScan(key, cursor, match, count, out items); } /// - public GarnetStatus HashTimeToLive(ArgSlice key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.Object); return garnetApi.HashTimeToLive(key, isMilliseconds, isTimestamp, ref input, ref output); @@ -521,45 +521,45 @@ public GarnetStatus HashTimeToLive(ArgSlice key, bool isMilliseconds, bool isTim #region Bitmap Methods /// - public GarnetStatus StringGetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + public GarnetStatus StringGetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.StringGetBit(ref key, ref input, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.StringGetBit(key, ref input, ref output); } /// - public GarnetStatus StringGetBit(ArgSlice key, ArgSlice offset, out bool bValue) + public GarnetStatus StringGetBit(PinnedSpanByte key, PinnedSpanByte offset, out bool bValue) { garnetApi.WATCH(key, StoreType.Main); return garnetApi.StringGetBit(key, offset, out bValue); } /// - public GarnetStatus StringBitCount(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + public GarnetStatus StringBitCount(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.StringBitCount(ref key, ref input, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.StringBitCount(key, ref input, ref output); } /// - public GarnetStatus StringBitCount(ArgSlice key, long start, long end, out long result, bool useBitInterval = false) + public GarnetStatus StringBitCount(PinnedSpanByte key, long start, long end, out long result, bool useBitInterval = false) { garnetApi.WATCH(key, StoreType.Main); return garnetApi.StringBitCount(key, start, end, out result, useBitInterval); } /// - public GarnetStatus StringBitPosition(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) + public GarnetStatus StringBitPosition(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.StringBitPosition(ref key, ref input, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.StringBitPosition(key, ref input, ref output); } /// - public GarnetStatus StringBitFieldReadOnly(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) + public GarnetStatus StringBitFieldReadOnly(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output) { - garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main); - return garnetApi.StringBitFieldReadOnly(ref key, ref input, secondaryCommand, ref output); + garnetApi.WATCH(key, StoreType.Main); + return garnetApi.StringBitFieldReadOnly(key, ref input, secondaryCommand, ref output); } #endregion @@ -579,7 +579,7 @@ public GarnetStatus HyperLogLogLength(ref RawStringInput input, out long count, } /// - public GarnetStatus HyperLogLogLength(Span keys, out long count) + public GarnetStatus HyperLogLogLength(Span keys, out long count) { foreach (var key in keys) { @@ -593,7 +593,7 @@ public GarnetStatus HyperLogLogLength(Span keys, out long count) #region Server Methods /// - public List GetDbKeys(ArgSlice pattern) + public List GetDbKeys(PinnedSpanByte pattern) { return garnetApi.GetDbKeys(pattern); } @@ -605,34 +605,34 @@ public int GetDbSize() } /// - public bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out long cursorStore, out List keys, long count = 10, ReadOnlySpan type = default) + public bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, out long cursorStore, out List keys, long count = 10, ReadOnlySpan type = default) { return garnetApi.DbScan(patternB, allKeys, cursor, out cursorStore, out keys, count, type); } /// public bool IterateMainStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => garnetApi.IterateMainStore(ref scanFunctions, untilAddress); /// - public ITsavoriteScanIterator IterateMainStore() + public ITsavoriteScanIterator IterateMainStore() => garnetApi.IterateMainStore(); /// public bool IterateObjectStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => garnetApi.IterateObjectStore(ref scanFunctions, untilAddress); /// - public ITsavoriteScanIterator IterateObjectStore() + public ITsavoriteScanIterator IterateObjectStore() => garnetApi.IterateObjectStore(); #endregion #region Common Methods - public GarnetStatus ObjectScan(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output) + public GarnetStatus ObjectScan(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output) { garnetApi.WATCH(key, StoreType.All); return garnetApi.ObjectScan(key, ref input, ref output); diff --git a/libs/server/API/IGarnetAdvancedApi.cs b/libs/server/API/IGarnetAdvancedApi.cs index 322f56948e8..e1ae3df275f 100644 --- a/libs/server/API/IGarnetAdvancedApi.cs +++ b/libs/server/API/IGarnetAdvancedApi.cs @@ -13,7 +13,7 @@ public interface IGarnetAdvancedApi /// /// GET with support for pending multiple ongoing operations, scatter gather IO for outputs /// - GarnetStatus GET_WithPending(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending); + GarnetStatus GET_WithPending(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending); /// /// Complete pending read operations on main store @@ -28,26 +28,26 @@ public interface IGarnetAdvancedApi /// /// /// - bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait = false); + bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait = false); /// /// RMW operation on main store /// - GarnetStatus RMW_MainStore(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus RMW_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Read operation on main store /// - GarnetStatus Read_MainStore(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus Read_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// RMW operation on object store /// - GarnetStatus RMW_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus RMW_ObjectStore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Read operation on object store /// - GarnetStatus Read_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus Read_ObjectStore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); } } \ No newline at end of file diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index ae1a1f30a74..733e5d33631 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -18,7 +18,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// GETEX /// - GarnetStatus GETEX(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus GETEX(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); #endregion @@ -26,51 +26,43 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// SET /// - GarnetStatus SET(ref SpanByte key, ref SpanByte value); + GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value); /// /// SET /// - GarnetStatus SET(ref SpanByte key, ref RawStringInput input, ref SpanByte value); + GarnetStatus SET(PinnedSpanByte key, ref RawStringInput input, PinnedSpanByte value); /// /// SET Conditional /// - GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input); + GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input); /// /// DEL Conditional /// - GarnetStatus DEL_Conditional(ref SpanByte key, ref RawStringInput input); + GarnetStatus DEL_Conditional(PinnedSpanByte key, ref RawStringInput input); /// /// SET Conditional /// - GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// SET /// - /// - /// - /// - GarnetStatus SET(ArgSlice key, Memory value); + GarnetStatus SET(PinnedSpanByte key, Memory value); /// /// SET /// - /// - /// - /// - GarnetStatus SET(ArgSlice key, ArgSlice value); + GarnetStatus SET(PinnedSpanByte key, IGarnetObject value); /// /// SET /// - /// - /// - /// - GarnetStatus SET(byte[] key, IGarnetObject value); + GarnetStatus SET(ref TSourceLogRecord srcLogRecord, StoreType storeType) + where TSourceLogRecord : ISourceLogRecord; #endregion #region SETEX @@ -81,7 +73,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Value /// Expiry in milliseconds, formatted as ASCII digits /// - GarnetStatus SETEX(ArgSlice key, ArgSlice value, ArgSlice expiryMs); + GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, PinnedSpanByte expiryMs); /// /// SETEX @@ -89,7 +81,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Key /// Value /// Expiry - GarnetStatus SETEX(ArgSlice key, ArgSlice value, TimeSpan expiry); + GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, TimeSpan expiry); #endregion @@ -102,7 +94,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// The output of the operation /// - GarnetStatus SETRANGE(ArgSlice key, ref RawStringInput input, ref ArgSlice output); + GarnetStatus SETRANGE(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output); #endregion @@ -125,7 +117,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// Length of updated value /// Operation status - GarnetStatus APPEND(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// APPEND command @@ -134,18 +126,15 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Value to be appended /// Length of updated value /// Operation status - GarnetStatus APPEND(ArgSlice key, ArgSlice value, ref ArgSlice output); + GarnetStatus APPEND(PinnedSpanByte key, PinnedSpanByte value, ref PinnedSpanByte output); #endregion #region RENAME /// /// RENAME /// - /// - /// - /// /// - GarnetStatus RENAME(ArgSlice oldKey, ArgSlice newKey, bool withEtag = false, StoreType storeType = StoreType.All); + GarnetStatus RENAME(PinnedSpanByte oldKey, PinnedSpanByte newKey, bool withEtag = false, StoreType storeType = StoreType.All); /// /// Renames key to newkey if newkey does not yet exist. It returns an error when key does not exist. @@ -153,9 +142,10 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// The old key to be renamed. /// The new key name. /// The result of the operation. + /// /// The type of store to perform the operation on. /// - GarnetStatus RENAMENX(ArgSlice oldKey, ArgSlice newKey, out int result, bool withEtag = false, StoreType storeType = StoreType.All); + GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out int result, bool withEtag = false, StoreType storeType = StoreType.All); #endregion #region EXISTS @@ -165,7 +155,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus EXISTS(ArgSlice key, StoreType storeType = StoreType.All); + GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All); #endregion #region EXPIRE @@ -178,7 +168,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIRE(ArgSlice key, ArgSlice expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); /// /// Set a timeout on key using a timeSpan in seconds @@ -188,7 +178,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Whether timeout was set by the call /// Store type: main, object, or both /// - GarnetStatus EXPIRE(ArgSlice key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All); + GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All); /// /// Set a timeout on key using a timeSpan in seconds @@ -199,7 +189,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIRE(ArgSlice key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); #endregion #region EXPIREAT @@ -213,7 +203,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Store type: main, object, or both /// Expire option /// - GarnetStatus EXPIREAT(ArgSlice key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); /// /// Set a timeout on key using absolute Unix timestamp (seconds since January 1, 1970) in milliseconds @@ -224,7 +214,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Store type: main, object, or both /// Expire option /// - GarnetStatus PEXPIREAT(ArgSlice key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); + GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None); #endregion @@ -235,7 +225,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Key /// Store type: main, object, or both /// - GarnetStatus PERSIST(ArgSlice key, StoreType storeType = StoreType.All); + GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType = StoreType.All); #endregion #region Increment (INCR, INCRBY, DECR, DECRBY) @@ -246,7 +236,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus Increment(ArgSlice key, ref RawStringInput input, ref ArgSlice output); + GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output); /// /// Increment (INCR, INCRBY) @@ -255,7 +245,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus Increment(ArgSlice key, out long output, long incrementCount = 1); + GarnetStatus Increment(PinnedSpanByte key, out long output, long incrementCount = 1); /// /// Decrement (DECR, DECRBY) @@ -264,7 +254,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus Decrement(ArgSlice key, out long output, long decrementCount = 1); + GarnetStatus Decrement(PinnedSpanByte key, out long output, long decrementCount = 1); #endregion #region DELETE @@ -274,23 +264,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus DELETE(ArgSlice key, StoreType storeType = StoreType.All); - - /// - /// DELETE - /// - /// - /// - /// - GarnetStatus DELETE(ref SpanByte key, StoreType storeType = StoreType.All); - - /// - /// DELETE - /// - /// - /// - /// - GarnetStatus DELETE(byte[] key, StoreType storeType = StoreType.All); + GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All); #endregion #region GETDEL @@ -300,15 +274,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Key to get and delete /// Current value of key /// Operation status - GarnetStatus GETDEL(ref SpanByte key, ref SpanByteAndMemory output); - - /// - /// GETDEL - /// - /// Key to get and delete - /// Current value of key - /// Operation status - GarnetStatus GETDEL(ArgSlice key, ref SpanByteAndMemory output); + GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output); #endregion #region TYPE @@ -320,7 +286,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus GetKeyType(ArgSlice key, out string typeName); + GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName); #endregion @@ -333,7 +299,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// The value in bytes the key or object is using /// Number of sampled nested values /// GarnetStatus - GarnetStatus MemoryUsageForKey(ArgSlice key, out long memoryUsage, int samples = 0); + GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, int samples = 0); #endregion @@ -347,7 +313,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Member /// Number of adds performed /// - GarnetStatus SortedSetAdd(ArgSlice key, ArgSlice score, ArgSlice member, out int zaddCount); + GarnetStatus SortedSetAdd(PinnedSpanByte key, PinnedSpanByte score, PinnedSpanByte member, out int zaddCount); /// /// Adds all the specified members with the specified scores to the sorted set stored at key. @@ -357,7 +323,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Input key-value pairs to add /// Number of adds performed /// - GarnetStatus SortedSetAdd(ArgSlice key, (ArgSlice score, ArgSlice member)[] inputs, out int zaddCount); + GarnetStatus SortedSetAdd(PinnedSpanByte key, (PinnedSpanByte score, PinnedSpanByte member)[] inputs, out int zaddCount); /// /// Adds all the specified members with the specified scores to the sorted set stored at key. @@ -367,7 +333,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Stores a range of sorted set elements in the specified key space. @@ -377,12 +343,12 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// The input object containing the elements to store. /// The result of the store operation. /// A indicating the status of the operation. - GarnetStatus SortedSetRangeStore(ArgSlice dstKey, ArgSlice srcKey, ref ObjectInput input, out int result); + GarnetStatus SortedSetRangeStore(PinnedSpanByte dstKey, PinnedSpanByte srcKey, ref ObjectInput input, out int result); /// /// Removes the specified member from the sorted set stored at key. /// - GarnetStatus SortedSetRemove(ArgSlice key, ArgSlice member, out int zremCount); + GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte member, out int zremCount); /// /// Removes the specified members from the sorted set stored at key. @@ -392,7 +358,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// Input members to remove /// Number of removes performed /// - GarnetStatus SortedSetRemove(ArgSlice key, ArgSlice[] members, out int zremCount); + GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int zremCount); /// /// Removes the specified members from the sorted set stored at key. @@ -402,7 +368,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Removes all elements in the sorted set between the @@ -412,7 +378,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemoveRangeByLex(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Removes and returns the first element from the sorted set stored at key, @@ -422,7 +388,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Removes and returns multiple elements from a sorted set. @@ -433,7 +399,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// The key of the popped element. /// An array of tuples containing the member and score of each popped element. /// A indicating the result of the operation. - GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out ArgSlice poppedKey, out (ArgSlice member, ArgSlice score)[] pairs); + GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out PinnedSpanByte poppedKey, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs); /// /// Removes and returns up to count members with the highest or lowest scores in the sorted set stored at key. @@ -443,7 +409,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// When true, return the members with the lowest scores, otherwise return the highest scores. /// - GarnetStatus SortedSetPop(ArgSlice key, out (ArgSlice member, ArgSlice score)[] pairs, int count = 1, bool lowScoresFirst = true); + GarnetStatus SortedSetPop(PinnedSpanByte key, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs, int count = 1, bool lowScoresFirst = true); /// /// Increments the score of member in the sorted set stored at key by increment. @@ -453,7 +419,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Increments the score of member in the sorted set stored at key by increment. @@ -465,7 +431,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetIncrement(ArgSlice key, Double increment, ArgSlice member, out double newScore); + GarnetStatus SortedSetIncrement(PinnedSpanByte key, double increment, PinnedSpanByte member, out double newScore); /// /// ZREMRANGEBYRANK: Removes all elements in the sorted set stored at key with rank between start and stop. @@ -476,7 +442,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemoveRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetRemoveRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Removes all elements in the range specified by min and max, having the same score. @@ -486,7 +452,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemoveRangeByLex(ArgSlice key, string min, string max, out int countRemoved); + GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, string min, string max, out int countRemoved); /// /// Removes all elements that have a score in the range specified by min and max. @@ -496,7 +462,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemoveRangeByScore(ArgSlice key, string min, string max, out int countRemoved); + GarnetStatus SortedSetRemoveRangeByScore(PinnedSpanByte key, string min, string max, out int countRemoved); /// /// Removes all elements with the index in the range specified by start and stop. @@ -506,7 +472,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetRemoveRangeByRank(ArgSlice key, int start, int stop, out int countRemoved); + GarnetStatus SortedSetRemoveRangeByRank(PinnedSpanByte key, int start, int stop, out int countRemoved); /// /// Computes the difference between the first and all successive sorted sets and store resulting pairs in the output key. @@ -515,7 +481,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySpan keys, out int count); + GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, out int count); /// /// Adds geospatial items (longitude, latitude, name) to the specified key. @@ -524,7 +490,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus GeoAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus GeoAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Geospatial search and store in destination key. @@ -535,7 +501,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi /// /// /// - GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearchOptions opts, + GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output); /// @@ -547,7 +513,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The type of aggregation to use for the intersection. /// The number of elements in the resulting sorted set. /// A indicating the status of the operation. - GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count); + GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count); /// /// Performs a union of multiple sorted sets and stores the result in the destination key. @@ -558,7 +524,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// Optional weights to apply to each sorted set. /// The type of aggregation to perform (e.g., Sum, Min, Max). /// A indicating the status of the operation. - GarnetStatus SortedSetUnionStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count); + GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count); /// /// Sets an expiration time on a sorted set member. @@ -567,7 +533,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus SortedSetExpire(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetExpire(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Sets an expiration time on a sorted set member. @@ -578,7 +544,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The expiration option to apply. /// The results of the operation. /// The status of the operation. - GarnetStatus SortedSetExpire(ArgSlice key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results); + GarnetStatus SortedSetExpire(PinnedSpanByte key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results); /// /// Persists the specified sorted set member, removing any expiration time set on it. @@ -587,7 +553,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus SortedSetPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Persists the specified sorted set members, removing any expiration time set on them. @@ -596,7 +562,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The members to persist. /// The results of the operation. /// The status of the operation. - GarnetStatus SortedSetPersist(ArgSlice key, ReadOnlySpan members, out int[] results); + GarnetStatus SortedSetPersist(PinnedSpanByte key, ReadOnlySpan members, out int[] results); /// /// Deletes already expired members from the sorted set. @@ -604,7 +570,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The keys of the sorted set members to check for expiration. /// The input object containing additional parameters. /// The status of the operation. - GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input); + GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input); /// /// Collects expired elements from the sorted set. @@ -617,7 +583,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// The keys of the sorted sets to collect expired elements from. /// The status of the operation. - GarnetStatus SortedSetCollect(ReadOnlySpan keys); + GarnetStatus SortedSetCollect(ReadOnlySpan keys); #endregion @@ -631,7 +597,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetAdd(ArgSlice key, ArgSlice member, out int saddCount); + GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte member, out int saddCount); /// /// Adds the specified members to the set at key. @@ -642,7 +608,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetAdd(ArgSlice key, ArgSlice[] members, out int saddCount); + GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int saddCount); /// /// Adds the specified members to the set at key. @@ -653,7 +619,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetAdd(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Removes the specified member from the set. @@ -664,7 +630,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetRemove(ArgSlice key, ArgSlice member, out int sremCount); + GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte member, out int sremCount); /// /// Removes the specified members from the set. @@ -675,7 +641,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetRemove(ArgSlice key, ArgSlice[] members, out int sremCount); + GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int sremCount); /// /// Removes the specified members from the set. @@ -686,7 +652,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Removes and returns one random member from the set at key. @@ -694,7 +660,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetPop(ArgSlice key, out ArgSlice member); + GarnetStatus SetPop(PinnedSpanByte key, out PinnedSpanByte member); /// /// Removes and returns random members from the set at key. @@ -703,7 +669,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetPop(ArgSlice key, int count, out ArgSlice[] members); + GarnetStatus SetPop(PinnedSpanByte key, int count, out PinnedSpanByte[] members); /// /// Removes and returns random members from the set at key. @@ -712,7 +678,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Moves a member from a source set to a destination set. @@ -724,7 +690,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey, ArgSlice member, out int smoveResult); + GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, PinnedSpanByte member, out int smoveResult); /// /// When called with just the key argument, return a random element from the set value stored at key. @@ -737,7 +703,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// This command is equal to SUNION, but instead of returning the resulting set, it is stored in destination. @@ -747,7 +713,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetUnionStore(byte[] key, ArgSlice[] keys, out int count); + GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count); /// /// This command is equal to SINTER, but instead of returning the resulting set, it is stored in destination. @@ -757,7 +723,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count); + GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count); /// /// This command is equal to SDIFF, but instead of returning the resulting set, it is stored in destination. @@ -767,7 +733,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - public GarnetStatus SetDiffStore(byte[] key, ArgSlice[] keys, out int count); + public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count); #endregion #region List Methods @@ -782,7 +748,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListPosition(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// ListLeftPush ArgSlice version with ObjectOutputHeader output @@ -791,7 +757,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListLeftPush(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// ListLeftPush ArgSlice version, one element @@ -801,7 +767,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// When true the operation is executed only if the key already exists /// - GarnetStatus ListLeftPush(ArgSlice key, ArgSlice element, out int count, bool whenExists = false); + GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte element, out int count, bool whenExists = false); /// /// ListLeftPush ArgSlice version for multiple values @@ -811,7 +777,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// When true the operation is executed only if the key already exists /// - GarnetStatus ListLeftPush(ArgSlice key, ArgSlice[] elements, out int count, bool whenExists = false); + GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int count, bool whenExists = false); /// /// ListRightPush ArgSlice version with ObjectOutputHeader output @@ -820,7 +786,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - public GarnetStatus ListRightPush(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// ListRightPush ArgSlice version, one element @@ -830,7 +796,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// When true the operation is executed only if the key already exists /// - GarnetStatus ListRightPush(ArgSlice key, ArgSlice element, out int count, bool whenExists = false); + GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte element, out int count, bool whenExists = false); /// /// ListRightPush ArgSlice version for multiple values @@ -840,7 +806,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// When true the operation is executed only if the key already exists /// - GarnetStatus ListRightPush(ArgSlice key, ArgSlice[] elements, out int count, bool whenExists = false); + GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int count, bool whenExists = false); #endregion @@ -853,7 +819,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListLeftPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListLeftPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// ListLeftPop ArgSlice version, one element @@ -861,7 +827,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListLeftPop(ArgSlice key, out ArgSlice element); + GarnetStatus ListLeftPop(PinnedSpanByte key, out PinnedSpanByte element); /// /// ListLeftPop ArgSlice version for multiple values @@ -870,7 +836,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListLeftPop(ArgSlice key, int count, out ArgSlice[] elements); + GarnetStatus ListLeftPop(PinnedSpanByte key, int count, out PinnedSpanByte[] elements); /// /// ListLeftPop ArgSlice version for multiple keys and values @@ -880,7 +846,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// GarnetStatus - GarnetStatus ListLeftPop(ArgSlice[] keys, int count, out ArgSlice key, out ArgSlice[] elements); + GarnetStatus ListLeftPop(PinnedSpanByte[] keys, int count, out PinnedSpanByte key, out PinnedSpanByte[] elements); /// /// ListRightPop ArgSlice version, with GarnetObjectStoreOutput @@ -889,7 +855,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListRightPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListRightPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// ListRightPop ArgSlice version, one element @@ -897,7 +863,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListRightPop(ArgSlice key, out ArgSlice element); + GarnetStatus ListRightPop(PinnedSpanByte key, out PinnedSpanByte element); /// /// ListRightPop ArgSlice version for multiple values @@ -906,7 +872,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListRightPop(ArgSlice key, int count, out ArgSlice[] elements); + GarnetStatus ListRightPop(PinnedSpanByte key, int count, out PinnedSpanByte[] elements); /// @@ -917,7 +883,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// GarnetStatus - GarnetStatus ListRightPop(ArgSlice[] keys, int count, out ArgSlice key, out ArgSlice[] elements); + GarnetStatus ListRightPop(PinnedSpanByte[] keys, int count, out PinnedSpanByte key, out PinnedSpanByte[] elements); #endregion @@ -931,7 +897,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// The element being popped and pushed /// GarnetStatus - public GarnetStatus ListMove(ArgSlice sourceKey, ArgSlice destinationKey, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element); + public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element); /// /// Trim an existing list so it only contains the specified range of elements. @@ -940,7 +906,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - public bool ListTrim(ArgSlice key, int start, int stop); + public bool ListTrim(PinnedSpanByte key, int start, int stop); /// /// Trim an existing list so it only contains the specified range of elements. @@ -948,7 +914,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListTrim(byte[] key, ref ObjectInput input); + GarnetStatus ListTrim(PinnedSpanByte key, ref ObjectInput input); /// /// Inserts a new element in the list stored at key either before or after a value pivot @@ -957,7 +923,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListInsert(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Removes the first count occurrences of elements equal to element from the list. @@ -966,7 +932,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Sets the list element at index to element. @@ -975,7 +941,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus ListSet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListSet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); #endregion @@ -989,7 +955,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashSet(ArgSlice key, ArgSlice field, ArgSlice value, out int count); + GarnetStatus HashSet(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int count); /// /// Sets the specified fields to their respective values in the hash stored at key. @@ -998,7 +964,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashSet(ArgSlice key, (ArgSlice field, ArgSlice value)[] elements, out int count); + GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpanByte value)[] elements, out int count); /// /// Sets or updates the values of the specified fields that exist in the hash. @@ -1011,7 +977,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashSet(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Set only if field does not yet exist. If key does not exist, a new key holding a hash is created. @@ -1023,7 +989,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashSetWhenNotExists(ArgSlice key, ArgSlice field, ArgSlice value, out int count); + GarnetStatus HashSetWhenNotExists(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int count); /// /// Removes the specified field from the hash stored at key. @@ -1032,7 +998,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// Number of fields removed /// - GarnetStatus HashDelete(ArgSlice key, ArgSlice field, out int count); + GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte field, out int count); /// /// Removes the specified fields from the hash stored at key. @@ -1041,7 +1007,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// Number of fields removed /// - GarnetStatus HashDelete(ArgSlice key, ArgSlice[] fields, out int count); + GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte[] fields, out int count); /// /// Removes the specified fields from the hash stored at key. @@ -1050,7 +1016,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashDelete(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Increments the number stored at field in the hash key by increment parameter. @@ -1059,7 +1025,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashIncrement(byte[] key, ArgSlice input, out ObjectOutputHeader output); + GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output); /// /// Increments the number stored at field representing a floating point value @@ -1069,18 +1035,19 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HashIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Sets an expiration time on a hash field. /// /// The key of the hash. /// The expiration time in Unix timestamp format. + /// /// The expiration option to apply. /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus HashExpire(ArgSlice key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashExpire(PinnedSpanByte key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Persists the specified hash key, removing any expiration time set on it. @@ -1089,7 +1056,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus HashPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Deletes already expired fields from the hash. @@ -1097,7 +1064,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// The keys of the hash fields to check for expiration. /// The input object containing additional parameters. /// The status of the operation. - GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input); + GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input); #endregion @@ -1111,7 +1078,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus StringSetBit(ArgSlice key, ArgSlice offset, bool bit, out bool previous); + GarnetStatus StringSetBit(PinnedSpanByte key, PinnedSpanByte offset, bool bit, out bool previous); /// /// Sets or clears the bit at offset in the given key. @@ -1122,7 +1089,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus StringSetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus StringSetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Performs a bitwise operations on multiple keys @@ -1142,7 +1109,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus StringBitOperation(BitmapOperation bitop, ArgSlice destinationKey, ArgSlice[] keys, out long result); + GarnetStatus StringBitOperation(BitmapOperation bitop, PinnedSpanByte destinationKey, PinnedSpanByte[] keys, out long result); /// /// Performs arbitrary bitfield integer operations on strings. @@ -1152,12 +1119,12 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus StringBitField(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output); + GarnetStatus StringBitField(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output); /// /// Performs arbitrary bitfield integer operations on strings. /// - GarnetStatus StringBitField(ArgSlice key, List commandArguments, out List result); + GarnetStatus StringBitField(PinnedSpanByte key, List commandArguments, out List result); #endregion #region HyperLogLog Methods @@ -1169,7 +1136,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// /// - GarnetStatus HyperLogLogAdd(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus HyperLogLogAdd(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Adds all the element arguments to the HyperLogLog data structure stored at the variable name specified as key. @@ -1178,7 +1145,7 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch /// /// true if at least 1 HyperLogLog internal register was altered /// - GarnetStatus HyperLogLogAdd(ArgSlice keys, string[] elements, out bool updated); + GarnetStatus HyperLogLogAdd(PinnedSpanByte keys, string[] elements, out bool updated); /// /// Merge multiple HyperLogLog values into a unique value that will approximate the cardinality @@ -1201,7 +1168,7 @@ public interface IGarnetReadApi /// /// GET /// - GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// GET @@ -1209,7 +1176,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GETForMemoryResult(ArgSlice key, out MemoryResult value); + GarnetStatus GETForMemoryResult(PinnedSpanByte key, out MemoryResult value); /// /// GET @@ -1217,7 +1184,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GET(ArgSlice key, out ArgSlice value); + GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte value); /// /// GET @@ -1225,7 +1192,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput value); + GarnetStatus GET(PinnedSpanByte key, out GarnetObjectStoreOutput value); /// /// Finds the longest common subsequence (LCS) between two keys. @@ -1238,7 +1205,7 @@ public interface IGarnetReadApi /// If true, the length of each match is returned. /// The minimum length of a match to be considered. /// The status of the operation. - GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0); + GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0); #endregion #region GETRANGE @@ -1250,7 +1217,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GETRANGE(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); #endregion #region TTL @@ -1262,7 +1229,7 @@ public interface IGarnetReadApi /// The store type to operate on. /// The span to allocate the output of the operation. /// - GarnetStatus TTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output); + GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); /// /// Returns the remaining time to live in milliseconds of a key that has a timeout. @@ -1271,7 +1238,7 @@ public interface IGarnetReadApi /// The store type to operate on. /// The span to allocate the output of the operation. /// - GarnetStatus PTTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output); + GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); #endregion @@ -1284,7 +1251,7 @@ public interface IGarnetReadApi /// The type of store to retrieve the key from. /// The output containing the expiration time. /// The status of the operation. - GarnetStatus EXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output); + 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. @@ -1293,7 +1260,7 @@ public interface IGarnetReadApi /// The type of store to retrieve the key from. /// The output containing the expiration time. /// The status of the operation. - GarnetStatus PEXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output); + GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output); #endregion @@ -1305,7 +1272,7 @@ public interface IGarnetReadApi /// Key /// /// - GarnetStatus SortedSetLength(ArgSlice key, out int zcardCount); + GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount); /// /// Returns the sorted set cardinality (number of elements) of the sorted set @@ -1314,7 +1281,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Returns the specified range of elements in the sorted set stored at key. @@ -1325,7 +1292,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the score of member in the sorted set at key. @@ -1335,7 +1302,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetScore(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetScore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the scores associated with the specified members in the sorted set stored at key. @@ -1345,7 +1312,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetScores(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetScores(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the number of elements in the sorted set at key with a score between min and max. @@ -1355,7 +1322,7 @@ public interface IGarnetReadApi /// Max score /// Number of elements /// - GarnetStatus SortedSetCount(ArgSlice key, ArgSlice minScore, ArgSlice maxScore, out int numElements); + GarnetStatus SortedSetCount(PinnedSpanByte key, PinnedSpanByte minScore, PinnedSpanByte maxScore, out int numElements); /// /// Returns the number of elements in the sorted set at key with a score between min and max. @@ -1364,7 +1331,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetCount(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the number of elements in the sorted set with a value between min and max. @@ -1375,7 +1342,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetLengthByValue(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// ZRANK: Returns the rank of member in the sorted set, the scores in the sorted set are ordered from low to high @@ -1385,7 +1352,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetRank(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetRank(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// ZRANK: Returns the rank of member in the sorted set, the scores in the sorted set are ordered from low to high @@ -1396,7 +1363,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetRank(ArgSlice key, ArgSlice member, bool reverse, out long? rank); + GarnetStatus SortedSetRank(PinnedSpanByte key, PinnedSpanByte member, bool reverse, out long? rank); /// /// Returns a random element from the sorted set key. @@ -1405,7 +1372,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the specified range of elements in the sorted set stored at key, using byscore, bylex and rev modifiers. @@ -1422,7 +1389,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetRange(ArgSlice key, ArgSlice min, ArgSlice max, SortedSetOrderOperation sortedSetOrderOperation, out ArgSlice[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default); + GarnetStatus SortedSetRange(PinnedSpanByte key, PinnedSpanByte min, PinnedSpanByte max, SortedSetOrderOperation sortedSetOrderOperation, out PinnedSpanByte[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default); /// /// Computes the difference between the first and all successive sorted sets and returns resulting pairs. @@ -1430,7 +1397,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus SortedSetDifference(ArgSlice[] keys, out Dictionary pairs); + GarnetStatus SortedSetDifference(PinnedSpanByte[] keys, out Dictionary pairs); /// /// Performs a union of multiple sorted sets and stores the result in a dictionary. @@ -1440,7 +1407,7 @@ public interface IGarnetReadApi /// An optional array of doubles representing the weights to apply to each sorted set during the union. /// The type of aggregation to use when combining scores from the sorted sets. Defaults to . /// A indicating the status of the operation. - GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs); + GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs); /// /// Iterates members of SortedSet key and their associated scores using a cursor, @@ -1452,7 +1419,7 @@ public interface IGarnetReadApi /// Limit number for the response /// The list of items for the response /// - GarnetStatus SortedSetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items); + GarnetStatus SortedSetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items); /// /// Intersects multiple sorted sets and returns the result. @@ -1462,7 +1429,7 @@ public interface IGarnetReadApi /// The type of aggregation to perform. /// The resulting dictionary of intersected elements and their scores. /// A indicating the status of the operation. - GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs); + GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs); /// /// Computes the intersection of multiple sorted sets and counts the elements. @@ -1471,7 +1438,7 @@ public interface IGarnetReadApi /// Optional max count limit /// The count of elements in the intersection /// Operation status - GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count); + GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count); /// /// Returns the time to live for a sorted set members. @@ -1480,7 +1447,7 @@ public interface IGarnetReadApi /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus SortedSetTimeToLive(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the time to live for a sorted set members. @@ -1489,7 +1456,7 @@ public interface IGarnetReadApi /// The members to get the time to live for. /// The output array containing the time to live for each member. /// The status of the operation. - GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySpan members, out TimeSpan[] expireIn); + GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ReadOnlySpan members, out TimeSpan[] expireIn); #endregion @@ -1504,7 +1471,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GeoCommands(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus GeoCommands(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// GEORADIUS (read variant): Return the members of a sorted set populated with geospatial data, which are inside the circular area delimited by center and radius. @@ -1518,8 +1485,7 @@ public interface IGarnetReadApi /// /// /// - GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, - ref ObjectInput input, ref SpanByteAndMemory output); + GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output); #endregion @@ -1531,7 +1497,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus ListLength(ArgSlice key, out int count); + GarnetStatus ListLength(PinnedSpanByte key, out int count); /// /// Gets length of the list, RESP version @@ -1540,7 +1506,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus ListLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Gets the specified elements of the list stored at key. @@ -1549,7 +1515,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus ListRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the element at index. @@ -1558,7 +1524,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus ListIndex(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); #endregion @@ -1570,7 +1536,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// Key /// /// - GarnetStatus SetLength(ArgSlice key, out int count); + GarnetStatus SetLength(PinnedSpanByte key, out int count); /// /// Returns the number of elements of the set. @@ -1579,7 +1545,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// SMEMBERS key @@ -1587,7 +1553,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetMembers(ArgSlice key, out ArgSlice[] members); + GarnetStatus SetMembers(PinnedSpanByte key, out PinnedSpanByte[] members); /// /// Returns all members of the set at key. @@ -1596,7 +1562,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetMembers(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns if member is a member of the set stored at key. @@ -1605,15 +1571,16 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetIsMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns whether each member is a member of the set stored at key. /// /// /// + /// /// - GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] members, out int[] result); + GarnetStatus SetIsMember(PinnedSpanByte key, PinnedSpanByte[] members, out int[] result); /// /// Iterates over the members of the Set with the given key using a cursor, @@ -1625,7 +1592,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items); + GarnetStatus SetScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items); /// /// Returns the members of the set resulting from the union of all the given sets. @@ -1634,7 +1601,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output); + GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output); /// /// Returns the members of the set resulting from the intersection of all the given sets. @@ -1643,7 +1610,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output); + GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet output); /// /// Returns the members of the set resulting from the difference between the first set and all the successive sets. @@ -1651,7 +1618,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus SetDiff(ArgSlice[] keys, out HashSet members); + GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet members); /// /// Returns the cardinality of the intersection between multiple sets. @@ -1661,7 +1628,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// Optional limit to stop counting at /// The cardinality of the intersection /// Operation status - GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count); + GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count); #endregion #region Hash Methods @@ -1673,7 +1640,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashGet(ArgSlice key, ArgSlice field, out ArgSlice value); + GarnetStatus HashGet(PinnedSpanByte key, PinnedSpanByte field, out PinnedSpanByte value); /// /// Returns the values associated with the fields in the hash stored at key. @@ -1682,7 +1649,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashGetMultiple(ArgSlice key, ArgSlice[] fields, out ArgSlice[] values); + GarnetStatus HashGetMultiple(PinnedSpanByte key, PinnedSpanByte[] fields, out PinnedSpanByte[] values); /// /// Returns the value associated with field in the hash stored at key. @@ -1691,7 +1658,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// The metadata input for the operation /// /// - GarnetStatus HashGet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns all fields and values of the hash stored at key. @@ -1700,7 +1667,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// The metadata input for the operation /// /// - GarnetStatus HashGetAll(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns the values associated with the specified fields in the hash stored at key. @@ -1709,7 +1676,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// The metadata input for the operation /// /// - GarnetStatus HashGetMultiple(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns ALL the values in the hash stored at key. @@ -1717,7 +1684,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashGetAll(ArgSlice key, out ArgSlice[] values); + GarnetStatus HashGetAll(PinnedSpanByte key, out PinnedSpanByte[] values); /// /// Returns the number of fields contained in the hash Key @@ -1725,7 +1692,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashLength(ArgSlice key, out int count); + GarnetStatus HashLength(PinnedSpanByte key, out int count); /// ///Returns the string length of the value associated with field in the hash stored at key. If the key or the field do not exist, 0 is returned. @@ -1734,7 +1701,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashStrLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Returns the number of fields contained in the hash Key. @@ -1743,7 +1710,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Returns if field is an existing field in the hash stored at key. @@ -1752,7 +1719,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashExists(ArgSlice key, ArgSlice field, out bool exists); + GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out bool exists); /// /// Returns if field is an existing field in the hash stored at key. @@ -1761,7 +1728,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashExists(byte[] key, ref ObjectInput input, out ObjectOutputHeader output); + GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output); /// /// Returns count random fields from the hash value. @@ -1771,7 +1738,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashRandomField(ArgSlice key, int count, bool withValues, out ArgSlice[] fields); + GarnetStatus HashRandomField(PinnedSpanByte key, int count, bool withValues, out PinnedSpanByte[] fields); /// /// Returns a random field from the hash value stored at key. @@ -1779,7 +1746,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashRandomField(ArgSlice key, out ArgSlice field); + GarnetStatus HashRandomField(PinnedSpanByte key, out PinnedSpanByte field); /// /// Returns a random field(s) from the hash value stored at key. @@ -1788,7 +1755,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashRandomField(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns all field names in the hash key. @@ -1797,7 +1764,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashKeys(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Returns all values in the hash key. @@ -1806,7 +1773,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashVals(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Iterates fields of Hash key and their associated values using a cursor, @@ -1818,7 +1785,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HashScan(ArgSlice key, long cursor, string match, int count, out ArgSlice[] items); + GarnetStatus HashScan(PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items); /// /// Returns the time to live for a hash key. @@ -1829,7 +1796,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// The input object containing additional parameters. /// The output object to store the result. /// The status of the operation. - GarnetStatus HashTimeToLive(ArgSlice key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output); #endregion @@ -1842,7 +1809,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringGetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus StringGetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Returns the bit value at offset in the key stored. @@ -1851,7 +1818,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringGetBit(ArgSlice key, ArgSlice offset, out bool bValue); + GarnetStatus StringGetBit(PinnedSpanByte key, PinnedSpanByte offset, out bool bValue); /// /// Count the number of set bits in a string. @@ -1861,7 +1828,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringBitCount(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus StringBitCount(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Count the number of set bits in a string. @@ -1873,7 +1840,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringBitCount(ArgSlice key, long start, long end, out long result, bool useBitInterval = false); + GarnetStatus StringBitCount(PinnedSpanByte key, long start, long end, out long result, bool useBitInterval = false); /// /// Returns the position of the first bit set to 1 or 0 in a key. @@ -1882,7 +1849,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringBitPosition(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); + GarnetStatus StringBitPosition(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output); /// /// Read-only variant of the StringBitField method. @@ -1892,7 +1859,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus StringBitFieldReadOnly(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output); + GarnetStatus StringBitFieldReadOnly(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output); #endregion @@ -1913,7 +1880,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// /// - GarnetStatus HyperLogLogLength(Span keys, out long count); + GarnetStatus HyperLogLogLength(Span keys, out long count); #endregion #region Server Methods @@ -1923,7 +1890,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// Expression to match the keys name /// - List GetDbKeys(ArgSlice pattern); + List GetDbKeys(PinnedSpanByte pattern); /// /// Gets the number of existing keys in both stores @@ -1942,7 +1909,7 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// 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, ReadOnlySpan type = default); + public bool DbScan(PinnedSpanByte patternB, bool allKeys, long cursor, out long storeCursor, out List Keys, long count = 10, ReadOnlySpan type = default); /// /// Iterate the contents of the main store @@ -1952,13 +1919,13 @@ GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, /// /// public bool IterateMainStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions; + where TScanFunctions : IScanIteratorFunctions; /// /// Iterate the contents of the main store (pull based) /// /// - public ITsavoriteScanIterator IterateMainStore(); + public ITsavoriteScanIterator IterateMainStore(); /// /// Iterate the contents of the object store @@ -1968,13 +1935,13 @@ public bool IterateMainStore(ref TScanFunctions scanFunctions, l /// /// public bool IterateObjectStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions; + where TScanFunctions : IScanIteratorFunctions; /// /// Iterate the contents of the object store (pull based) /// /// - public ITsavoriteScanIterator IterateObjectStore(); + public ITsavoriteScanIterator IterateObjectStore(); #endregion @@ -1987,7 +1954,7 @@ public bool IterateObjectStore(ref TScanFunctions scanFunctions, /// The key of the sorted set /// /// - GarnetStatus ObjectScan(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output); + GarnetStatus ObjectScan(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output); /// /// Retrieve the current scratch buffer offset. @@ -2016,13 +1983,6 @@ public interface IGarnetWatchApi /// /// /// - void WATCH(ArgSlice key, StoreType type); - - /// - /// WATCH - /// - /// - /// - void WATCH(byte[] key, StoreType type); + void WATCH(PinnedSpanByte key, StoreType type); } } \ No newline at end of file diff --git a/libs/server/ArgSlice/ArgSlice.cs b/libs/server/ArgSlice/ArgSlice.cs deleted file mode 100644 index 8ea0a889306..00000000000 --- a/libs/server/ArgSlice/ArgSlice.cs +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; -using System.Text; -using Tsavorite.core; - -namespace Garnet.server -{ - /// - /// Represents contiguous region of arbitrary _pinned_ memory. - /// - /// - /// SAFETY: This type is used to represent arguments that are assumed to point to pinned memory. - /// - [StructLayout(LayoutKind.Explicit, Size = Size)] - public unsafe struct ArgSlice - { - public const int Size = 12; - - [FieldOffset(0)] - internal byte* ptr; - - [FieldOffset(8)] - internal int length; - - /// - /// Create new ArgSlice from given pointer and length - /// - public ArgSlice(byte* ptr, int length) - { - this.ptr = ptr; - this.length = length; - } - - /// - /// Create new ArgSlice from given SpanByte (without metadata header) - /// - internal ArgSlice(ref SpanByte input) - { - this.ptr = input.ToPointer(); - this.length = input.LengthWithoutMetadata; - } - - /// - /// Get length of ArgSlice - /// - public readonly int Length => length; - - /// - /// Get slice as ReadOnlySpan - /// - public readonly ReadOnlySpan ReadOnlySpan => new(ptr, length); - - /// - /// Get slice as Span - /// - public readonly Span Span => new(ptr, length); - - /// - /// Get slice as SpanByte - /// - public readonly SpanByte SpanByte => new(length, (nint)ptr); - - /// - /// Copies the contents of this slice into a new array. - /// - public readonly byte[] ToArray() => ReadOnlySpan.ToArray(); - - /// - /// Decodes the contents of this slice as ASCII into a new string. - /// - /// A string ASCII decoded string from the slice. - public override readonly string ToString() - => Encoding.ASCII.GetString(ReadOnlySpan); - - /// - /// Create a from the given . - /// - /// - /// SAFETY: The MUST point to pinned memory. - /// - internal static ArgSlice FromPinnedSpan(ReadOnlySpan span) - { - return new ArgSlice((byte*)Unsafe.AsPointer(ref MemoryMarshal.GetReference(span)), span.Length); - } - - /// - /// Check for equality to the provided argSlice - /// - /// - /// - public readonly bool Equals(ArgSlice argSlice) => argSlice.Span.SequenceEqual(Span); - } -} \ No newline at end of file diff --git a/libs/server/ArgSlice/ArgSliceUtils.cs b/libs/server/ArgSlice/ArgSliceUtils.cs deleted file mode 100644 index b418ef77cad..00000000000 --- a/libs/server/ArgSlice/ArgSliceUtils.cs +++ /dev/null @@ -1,19 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using Garnet.common; - -namespace Garnet.server -{ - /// - /// ArgSlice utils - /// - public static class ArgSliceUtils - { - /// - /// Compute hash slot of given ArgSlice - /// - public static unsafe ushort HashSlot(ref ArgSlice argSlice) - => HashSlotUtils.HashSlot(argSlice.ptr, argSlice.Length); - } -} \ No newline at end of file diff --git a/libs/server/ByteArrayWrapper.cs b/libs/server/ByteArrayWrapper.cs index 749f4fe53d1..c48034b6439 100644 --- a/libs/server/ByteArrayWrapper.cs +++ b/libs/server/ByteArrayWrapper.cs @@ -3,6 +3,7 @@ using System; using System.Numerics; +using Tsavorite.core; namespace Garnet.server { @@ -12,18 +13,16 @@ namespace Garnet.server public readonly struct ByteArrayWrapper { readonly byte[] arrBytes; - readonly ArgSlice arrSlice; + readonly PinnedSpanByte arrSlice; internal ByteArrayWrapper(byte[] arrBytes, bool isPinned = false) { this.arrBytes = arrBytes; if (isPinned) - { - this.arrSlice = ArgSlice.FromPinnedSpan(arrBytes); - } + this.arrSlice = PinnedSpanByte.FromPinnedSpan(arrBytes); } - internal ByteArrayWrapper(ArgSlice arrSlice) + internal ByteArrayWrapper(PinnedSpanByte arrSlice) { this.arrSlice = arrSlice; } @@ -36,22 +35,16 @@ public static ByteArrayWrapper CopyFrom(ReadOnlySpan bytes, bool usePinned } public unsafe ReadOnlySpan ReadOnlySpan - => arrSlice.ptr == null ? new ReadOnlySpan(arrBytes) : arrSlice.ReadOnlySpan; + => arrSlice.IsValid ? arrSlice.ReadOnlySpan : new ReadOnlySpan(arrBytes); /// public override unsafe int GetHashCode() { - if (arrSlice.ptr == null) - { - fixed (byte* k = arrBytes) - { - return (int)HashBytes(k, arrBytes.Length); - } - } - else - { - return (int)HashBytes(arrSlice.ptr, arrSlice.length); - } + if (arrSlice.IsValid) + return (int)HashBytes(arrSlice.ToPointer(), arrSlice.Length); + + fixed (byte* k = arrBytes) + return (int)HashBytes(k, arrBytes.Length); } static unsafe long HashBytes(byte* pbString, int len) diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs index 4a40b0a558e..564a62409b0 100644 --- a/libs/server/Cluster/IClusterProvider.cs +++ b/libs/server/Cluster/IClusterProvider.cs @@ -12,12 +12,12 @@ namespace Garnet.server { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; /// /// Cluster provider diff --git a/libs/server/Cluster/IClusterSession.cs b/libs/server/Cluster/IClusterSession.cs index 7fe00f9bc4b..4178e7c2142 100644 --- a/libs/server/Cluster/IClusterSession.cs +++ b/libs/server/Cluster/IClusterSession.cs @@ -4,6 +4,7 @@ using System; using Garnet.common; using Garnet.server.ACL; +using Tsavorite.core; namespace Garnet.server { @@ -65,7 +66,7 @@ public interface IClusterSession /// /// /// - bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking); + bool NetworkIterativeSlotVerify(PinnedSpanByte keySlice, bool readOnly, byte SessionAsking); /// /// Write cached slot verification message to output @@ -76,7 +77,7 @@ public interface IClusterSession /// /// Key array slot verify (write result to network) /// - unsafe bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte SessionAsking, ref byte* dcurr, ref byte* dend, int count = -1); + unsafe bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte SessionAsking, ref byte* dcurr, ref byte* dend, int count = -1); /// /// Array slot verify (write result to network) diff --git a/libs/server/Custom/CustomFunctions.cs b/libs/server/Custom/CustomFunctions.cs index 1aafef8b963..3613ec57419 100644 --- a/libs/server/Custom/CustomFunctions.cs +++ b/libs/server/Custom/CustomFunctions.cs @@ -6,6 +6,7 @@ using System.Collections.Generic; using System.Diagnostics; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -63,7 +64,7 @@ protected static unsafe void WriteSimpleString(ref MemoryResult output, Re /// /// Create output as an array of bulk strings, from given array of ArgSlice values /// - protected static unsafe void WriteBulkStringArray(ref MemoryResult output, params ArgSlice[] values) + protected static unsafe void WriteBulkStringArray(ref MemoryResult output, params PinnedSpanByte[] values) { var totalLen = 1 + NumUtils.CountDigits(values.Length) + 2; for (var i = 0; i < values.Length; i++) @@ -91,7 +92,7 @@ protected static unsafe void WriteBulkStringArray(ref MemoryResult output, /// /// Create output as an array of bulk strings, from given array of ArgSlice values /// - protected static unsafe void WriteBulkStringArray(ref MemoryResult output, List values) + protected static unsafe void WriteBulkStringArray(ref MemoryResult output, List values) { var totalLen = 1 + NumUtils.CountDigits(values.Count) + 2; for (var i = 0; i < values.Count; i++) @@ -201,7 +202,7 @@ protected static unsafe void WriteError(ref MemoryResult output, ReadOnlyS /// Current parse state /// Current argument index in parse state /// Argument as a span - protected static unsafe ArgSlice GetNextArg(ref SessionParseState parseState, ref int idx) + protected static unsafe PinnedSpanByte GetNextArg(ref SessionParseState parseState, ref int idx) { var arg = idx < parseState.Count ? parseState.GetArgSliceByRef(idx) @@ -216,7 +217,7 @@ protected static unsafe ArgSlice GetNextArg(ref SessionParseState parseState, re /// Procedure input /// Current argument index in parse state /// Argument as a span - protected static unsafe ArgSlice GetNextArg(ref CustomProcedureInput procInput, ref int idx) + protected static unsafe PinnedSpanByte GetNextArg(ref CustomProcedureInput procInput, ref int idx) { return GetNextArg(ref procInput.parseState, ref idx); } @@ -243,7 +244,7 @@ protected bool ParseCustomObjectCommand(string cmd, out CustomObjectCommand obje /// Args to the command /// Output from the command /// True if successful - protected bool ExecuteCustomRawStringCommand(TGarnetApi garnetApi, CustomRawStringCommand rawStringCommand, ArgSlice key, ArgSlice[] input, out ArgSlice output) + protected bool ExecuteCustomRawStringCommand(TGarnetApi garnetApi, CustomRawStringCommand rawStringCommand, PinnedSpanByte key, PinnedSpanByte[] input, out PinnedSpanByte output) where TGarnetApi : IGarnetApi { return respServerSession.InvokeCustomRawStringCommand(ref garnetApi, rawStringCommand, key, input, out output); @@ -257,7 +258,7 @@ protected bool ExecuteCustomRawStringCommand(TGarnetApi garnetApi, C /// Args to the command /// Output from the command /// True if successful - protected bool ExecuteCustomObjectCommand(TGarnetApi garnetApi, CustomObjectCommand objectCommand, ArgSlice key, ArgSlice[] input, out ArgSlice output) + protected bool ExecuteCustomObjectCommand(TGarnetApi garnetApi, CustomObjectCommand objectCommand, PinnedSpanByte key, PinnedSpanByte[] input, out PinnedSpanByte output) where TGarnetApi : IGarnetApi { return respServerSession.InvokeCustomObjectCommand(ref garnetApi, objectCommand, key, input, out output); diff --git a/libs/server/Custom/CustomObjectBase.cs b/libs/server/Custom/CustomObjectBase.cs index a659ad4418f..81e354f914c 100644 --- a/libs/server/Custom/CustomObjectBase.cs +++ b/libs/server/Custom/CustomObjectBase.cs @@ -18,15 +18,15 @@ public abstract class CustomObjectBase : GarnetObjectBase /// Base constructor /// /// Object type - /// - protected CustomObjectBase(byte type, long expiration, long size = 0) - : base(expiration, size) + /// + protected CustomObjectBase(byte type, ObjectSizes sizes = default) + : base(sizes) { this.type = type; } - protected CustomObjectBase(byte type, BinaryReader reader, long size = 0) - : base(reader, size) + protected CustomObjectBase(byte type, BinaryReader reader, ObjectSizes sizes = default) + : base(reader, sizes) { this.type = type; } @@ -35,7 +35,7 @@ protected CustomObjectBase(byte type, BinaryReader reader, long size = 0) /// Base copy constructor /// /// Other object - protected CustomObjectBase(CustomObjectBase obj) : this(obj.type, obj.Expiration, obj.Size) { } + protected CustomObjectBase(CustomObjectBase obj) : this(obj.type, obj.sizes) { } /// public override byte Type => type; diff --git a/libs/server/Custom/CustomObjectFunctions.cs b/libs/server/Custom/CustomObjectFunctions.cs index 630a323cd3e..3c6d05a15fa 100644 --- a/libs/server/Custom/CustomObjectFunctions.cs +++ b/libs/server/Custom/CustomObjectFunctions.cs @@ -50,7 +50,7 @@ public abstract class CustomObjectFunctions /// Key /// Input /// Output - public virtual bool NeedInitialUpdate(ReadOnlyMemory key, ref ObjectInput input, ref (IMemoryOwner, int) output) => throw new NotImplementedException(); + public virtual bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref (IMemoryOwner, int) output) => throw new NotImplementedException(); /// /// Create initial value, given key and input. Optionally generate output for command. @@ -61,7 +61,7 @@ public abstract class CustomObjectFunctions /// Output /// Advanced arguments /// True if done, false if we need to cancel the update - public virtual bool InitialUpdater(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) => Updater(key, ref input, value, ref output, ref rmwInfo); + public virtual bool InitialUpdater(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) => Updater(key, ref input, value, ref output, ref rmwInfo); /// /// Update given value in place, given key and input. Optionally generate output for command. @@ -72,7 +72,7 @@ public abstract class CustomObjectFunctions /// Output /// Advanced arguments /// True if done, false if we have no space to update in place - public virtual bool Updater(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) => throw new NotImplementedException(); + public virtual bool Updater(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) => throw new NotImplementedException(); /// /// Read value, given key and input and generate output for command. @@ -83,6 +83,6 @@ public abstract class CustomObjectFunctions /// Output /// Advanced arguments /// True if done, false if not found - public virtual bool Reader(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) => throw new NotImplementedException(); + public virtual bool Reader(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) => throw new NotImplementedException(); } } \ No newline at end of file diff --git a/libs/server/Custom/CustomRespCommands.cs b/libs/server/Custom/CustomRespCommands.cs index 36e767396ad..64ce3a92b14 100644 --- a/libs/server/Custom/CustomRespCommands.cs +++ b/libs/server/Custom/CustomRespCommands.cs @@ -87,16 +87,15 @@ private void TryCustomProcedure(CustomProcedure proc, int startIdx = 0) private bool TryCustomRawStringCommand(RespCommand cmd, long expirationTicks, CommandType type, ref TGarnetApi storageApi) where TGarnetApi : IGarnetAdvancedApi { - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); var inputArg = expirationTicks > 0 ? DateTimeOffset.UtcNow.Ticks + expirationTicks : expirationTicks; var input = new RawStringInput(cmd, ref parseState, startIdx: 1, arg1: inputArg); var output = new SpanByteAndMemory(null); - GarnetStatus status; if (type == CommandType.ReadModifyWrite) { - status = storageApi.RMW_MainStore(ref sbKey, ref input, ref output); + _ = storageApi.RMW_MainStore(key, ref input, ref output); Debug.Assert(!output.IsSpanByte); if (output.Memory != null) @@ -107,7 +106,7 @@ private bool TryCustomRawStringCommand(RespCommand cmd, long expirat } else { - status = storageApi.Read_MainStore(ref sbKey, ref input, ref output); + var status = storageApi.Read_MainStore(key, ref input, ref output); Debug.Assert(!output.IsSpanByte); if (status == GarnetStatus.OK) @@ -134,7 +133,7 @@ private bool TryCustomRawStringCommand(RespCommand cmd, long expirat private bool TryCustomObjectCommand(GarnetObjectType objType, byte subid, CommandType type, ref TGarnetApi storageApi) where TGarnetApi : IGarnetAdvancedApi { - var keyBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input @@ -147,7 +146,7 @@ private bool TryCustomObjectCommand(GarnetObjectType objType, byte s if (type == CommandType.ReadModifyWrite) { - status = storageApi.RMW_ObjectStore(ref keyBytes, ref input, ref output); + status = storageApi.RMW_ObjectStore(key, ref input, ref output); Debug.Assert(!output.SpanByteAndMemory.IsSpanByte); switch (status) @@ -167,7 +166,7 @@ private bool TryCustomObjectCommand(GarnetObjectType objType, byte s } else { - status = storageApi.Read_ObjectStore(ref keyBytes, ref input, ref output); + status = storageApi.Read_ObjectStore(key, ref input, ref output); Debug.Assert(!output.SpanByteAndMemory.IsSpanByte); switch (status) @@ -215,27 +214,25 @@ public bool ParseCustomObjectCommand(string cmd, out CustomObjectCommand customO /// Args to the command /// Output from the command /// True if successful - public bool InvokeCustomRawStringCommand(ref TGarnetApi storageApi, CustomRawStringCommand customCommand, ArgSlice key, ArgSlice[] args, out ArgSlice output) + public bool InvokeCustomRawStringCommand(ref TGarnetApi storageApi, CustomRawStringCommand customCommand, PinnedSpanByte key, PinnedSpanByte[] args, out PinnedSpanByte output) where TGarnetApi : IGarnetAdvancedApi { ArgumentNullException.ThrowIfNull(customCommand); - var sbKey = key.SpanByte; var inputArg = customCommand.expirationTicks > 0 ? DateTimeOffset.UtcNow.Ticks + customCommand.expirationTicks : customCommand.expirationTicks; customCommandParseState.InitializeWithArguments(args); var cmd = customCommandManagerSession.GetCustomRespCommand(customCommand.id); var rawStringInput = new RawStringInput(cmd, ref customCommandParseState, arg1: inputArg); var _output = new SpanByteAndMemory(null); - GarnetStatus status; if (customCommand.type == CommandType.ReadModifyWrite) { - status = storageApi.RMW_MainStore(ref sbKey, ref rawStringInput, ref _output); + _ = storageApi.RMW_MainStore(key, ref rawStringInput, ref _output); Debug.Assert(!_output.IsSpanByte); if (_output.Memory != null) { - output = scratchBufferManager.FormatScratch(0, _output.AsReadOnlySpan()); + output = scratchBufferManager.FormatScratch(0, _output.ReadOnlySpan); _output.Memory.Dispose(); } else @@ -245,14 +242,14 @@ public bool InvokeCustomRawStringCommand(ref TGarnetApi storageApi, } else { - status = storageApi.Read_MainStore(ref sbKey, ref rawStringInput, ref _output); + var status = storageApi.Read_MainStore(key, ref rawStringInput, ref _output); Debug.Assert(!_output.IsSpanByte); if (status == GarnetStatus.OK) { if (_output.Memory != null) { - output = scratchBufferManager.FormatScratch(0, _output.AsReadOnlySpan()); + output = scratchBufferManager.FormatScratch(0, _output.ReadOnlySpan); _output.Memory.Dispose(); } else @@ -281,15 +278,13 @@ public bool InvokeCustomRawStringCommand(ref TGarnetApi storageApi, /// Args to the command /// Output from the command /// True if successful - public bool InvokeCustomObjectCommand(ref TGarnetApi storageApi, CustomObjectCommand customObjCommand, ArgSlice key, ArgSlice[] args, out ArgSlice output) + public bool InvokeCustomObjectCommand(ref TGarnetApi storageApi, CustomObjectCommand customObjCommand, PinnedSpanByte key, PinnedSpanByte[] args, out PinnedSpanByte output) where TGarnetApi : IGarnetAdvancedApi { ArgumentNullException.ThrowIfNull(customObjCommand); output = default; - var keyBytes = key.ToArray(); - // Prepare input var type = customCommandManagerSession.GetCustomGarnetObjectType(customObjCommand.id); var header = new RespInputHeader(type) { SubId = customObjCommand.subid }; @@ -300,7 +295,7 @@ public bool InvokeCustomObjectCommand(ref TGarnetApi storageApi, Cus GarnetStatus status; if (customObjCommand.type == CommandType.ReadModifyWrite) { - status = storageApi.RMW_ObjectStore(ref keyBytes, ref input, ref _output); + status = storageApi.RMW_ObjectStore(key, ref input, ref _output); Debug.Assert(!_output.SpanByteAndMemory.IsSpanByte); switch (status) @@ -310,7 +305,7 @@ public bool InvokeCustomObjectCommand(ref TGarnetApi storageApi, Cus break; default: if (_output.SpanByteAndMemory.Memory != null) - output = scratchBufferManager.FormatScratch(0, _output.SpanByteAndMemory.AsReadOnlySpan()); + output = scratchBufferManager.FormatScratch(0, _output.SpanByteAndMemory.ReadOnlySpan); else output = scratchBufferManager.CreateArgSlice(CmdStrings.RESP_OK); break; @@ -318,14 +313,14 @@ public bool InvokeCustomObjectCommand(ref TGarnetApi storageApi, Cus } else { - status = storageApi.Read_ObjectStore(ref keyBytes, ref input, ref _output); + status = storageApi.Read_ObjectStore(key, ref input, ref _output); Debug.Assert(!_output.SpanByteAndMemory.IsSpanByte); switch (status) { case GarnetStatus.OK: if (_output.SpanByteAndMemory.Memory != null) - output = scratchBufferManager.FormatScratch(0, _output.SpanByteAndMemory.AsReadOnlySpan()); + output = scratchBufferManager.FormatScratch(0, _output.SpanByteAndMemory.ReadOnlySpan); else output = scratchBufferManager.CreateArgSlice(CmdStrings.RESP_OK); break; diff --git a/libs/server/Custom/CustomTransactionProcedure.cs b/libs/server/Custom/CustomTransactionProcedure.cs index 393ce40d5e3..2ff41506ab5 100644 --- a/libs/server/Custom/CustomTransactionProcedure.cs +++ b/libs/server/Custom/CustomTransactionProcedure.cs @@ -37,7 +37,7 @@ public abstract class CustomTransactionProcedure : CustomFunctions /// /// /// - protected void AddKey(ArgSlice key, LockType type, bool isObject) + protected void AddKey(PinnedSpanByte key, LockType type, bool isObject) { txnManager.SaveKeyEntryToLock(key, isObject, type); txnManager.VerifyKeyOwnership(key, type); @@ -47,19 +47,19 @@ protected void AddKey(ArgSlice key, LockType type, bool isObject) /// Rewind (pop) the last entry of scratch buffer (rewinding the current scratch buffer offset), /// if it contains the given ArgSlice /// - protected bool RewindScratchBuffer(ref ArgSlice slice) - => scratchBufferManager.RewindScratchBuffer(ref slice); + protected bool RewindScratchBuffer(PinnedSpanByte slice) + => scratchBufferManager.RewindScratchBuffer(slice); /// /// Create ArgSlice in scratch buffer, from given ReadOnlySpan /// - protected ArgSlice CreateArgSlice(ReadOnlySpan bytes) + protected PinnedSpanByte CreateArgSlice(ReadOnlySpan bytes) => scratchBufferManager.CreateArgSlice(bytes); /// /// Create ArgSlice in UTF8 format in scratch buffer, from given string /// - protected ArgSlice CreateArgSlice(string str) + protected PinnedSpanByte CreateArgSlice(string str) => scratchBufferManager.CreateArgSlice(str); /// diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs index ca7501e04b1..82bb3f420de 100644 --- a/libs/server/Databases/DatabaseManagerBase.cs +++ b/libs/server/Databases/DatabaseManagerBase.cs @@ -9,11 +9,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Base class for logical database management @@ -113,10 +113,10 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, public abstract IDatabaseManager Clone(bool enableAof); /// - public TsavoriteKV MainStore => DefaultDatabase.MainStore; + public TsavoriteKV MainStore => DefaultDatabase.MainStore; /// - public TsavoriteKV ObjectStore => DefaultDatabase.ObjectStore; + public TsavoriteKV ObjectStore => DefaultDatabase.ObjectStore; /// public TsavoriteLog AppendOnlyFile => DefaultDatabase.AppendOnlyFile; @@ -479,7 +479,7 @@ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objec break; case LogCompactionType.Scan: - mainStoreLog.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Scan); + mainStoreLog.Compact(untilAddress, CompactionType.Scan); if (compactionForceDelete) { CompactionCommitAof(db); @@ -488,7 +488,7 @@ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objec break; case LogCompactionType.Lookup: - mainStoreLog.Compact>(new SpanByteFunctions(), untilAddress, CompactionType.Lookup); + mainStoreLog.Compact(untilAddress, CompactionType.Lookup); if (compactionForceDelete) { CompactionCommitAof(db); @@ -524,8 +524,7 @@ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objec break; case LogCompactionType.Scan: - objectStoreLog.Compact>( - new SimpleSessionFunctions(), untilAddress, CompactionType.Scan); + objectStoreLog.Compact(untilAddress, CompactionType.Scan); if (compactionForceDelete) { CompactionCommitAof(db); @@ -534,8 +533,7 @@ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objec break; case LogCompactionType.Lookup: - objectStoreLog.Compact>( - new SimpleSessionFunctions(), untilAddress, CompactionType.Lookup); + objectStoreLog.Compact(untilAddress, CompactionType.Lookup); if (compactionForceDelete) { CompactionCommitAof(db); @@ -644,11 +642,12 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo // During the checkpoint, we may have serialized Garnet objects in (v) versions of objects. // We can now safely remove these serialized versions as they are no longer needed. using var iter1 = db.ObjectStore.Log.Scan(db.ObjectStore.Log.ReadOnlyAddress, - db.ObjectStore.Log.TailAddress, ScanBufferingMode.SinglePageBuffering, includeSealedRecords: true); - while (iter1.GetNext(out _, out _, out var value)) + db.ObjectStore.Log.TailAddress, DiskScanBufferingMode.SinglePageBuffering, includeSealedRecords: true); + while (iter1.GetNext()) { - if (value != null) - ((GarnetObjectBase)value).serialized = null; + var valueObject = iter1.ValueObject; + if (valueObject != null) + ((GarnetObjectBase)iter1.ValueObject).serialized = null; } } diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs index b7ae3774606..de3bd5f2361 100644 --- a/libs/server/Databases/IDatabaseManager.cs +++ b/libs/server/Databases/IDatabaseManager.cs @@ -9,11 +9,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Interface for logical database management @@ -28,12 +28,12 @@ public interface IDatabaseManager : IDisposable /// /// Store (of DB 0) /// - public TsavoriteKV MainStore { get; } + public TsavoriteKV MainStore { get; } /// /// Object store (of DB 0) /// - public TsavoriteKV ObjectStore { get; } + public TsavoriteKV ObjectStore { get; } /// /// AOF (of DB 0) diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs index dd5e00e2540..1832be9918a 100644 --- a/libs/server/Databases/MultiDatabaseManager.cs +++ b/libs/server/Databases/MultiDatabaseManager.cs @@ -642,8 +642,8 @@ 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.customCommandManager, null, db.ObjectStoreSizeTracker, - StoreWrapper.GarnetObjectSerializer, respProtocolVersion); + return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper.customCommandManager, memoryPool: null, db.ObjectStoreSizeTracker, + StoreWrapper.GarnetObjectSerializer, Logger, respProtocolVersion); } /// diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs index 1064083dc0e..a78afa04c31 100644 --- a/libs/server/Databases/SingleDatabaseManager.cs +++ b/libs/server/Databases/SingleDatabaseManager.cs @@ -368,7 +368,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0); return new(AppendOnlyFile, VersionMap, StoreWrapper.customCommandManager, null, ObjectStoreSizeTracker, - StoreWrapper.GarnetObjectSerializer, respProtocolVersion); + StoreWrapper.GarnetObjectSerializer, Logger, respProtocolVersion); } private async Task TryPauseCheckpointsContinuousAsync(int dbId, diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs index 4772fa2a276..8db4e1a23a6 100644 --- a/libs/server/GarnetDatabase.cs +++ b/libs/server/GarnetDatabase.cs @@ -5,11 +5,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Represents a logical database in Garnet @@ -30,12 +30,12 @@ public class GarnetDatabase : IDisposable /// /// Main Store /// - public TsavoriteKV MainStore { get; } + public TsavoriteKV MainStore { get; } /// /// Object Store /// - public TsavoriteKV ObjectStore { get; } + public TsavoriteKV ObjectStore { get; } /// /// Epoch instance used by server @@ -99,8 +99,8 @@ public class GarnetDatabase : IDisposable bool disposed = false; - public GarnetDatabase(int id, TsavoriteKV mainStore, - TsavoriteKV objectStore, + public GarnetDatabase(int id, TsavoriteKV mainStore, + TsavoriteKV objectStore, LightEpoch epoch, StateMachineDriver stateMachineDriver, CacheSizeTracker objectStoreSizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile, bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut) : this() diff --git a/libs/server/InputHeader.cs b/libs/server/InputHeader.cs index 1b824259563..cb7ac58096a 100644 --- a/libs/server/InputHeader.cs +++ b/libs/server/InputHeader.cs @@ -138,18 +138,13 @@ internal ListOperation ListOp /// /// Set "WithEtag" flag for the input header /// - internal void SetWithEtagFlag() => flags |= RespInputFlags.WithEtag; + internal void SetWithETagFlag() => flags |= RespInputFlags.WithEtag; /// /// Check if the WithEtag flag is set /// /// - internal bool CheckWithEtagFlag() => (flags & RespInputFlags.WithEtag) != 0; - - /// - /// Check that neither SetGet nor WithEtag flag is set - /// - internal bool NotSetGetNorCheckWithEtag() => (flags & (RespInputFlags.SetGet | RespInputFlags.WithEtag)) == 0; + internal bool CheckWithETagFlag() => (flags & RespInputFlags.WithEtag) != 0; /// /// Check if record is expired, either deterministically during log replay, @@ -157,22 +152,10 @@ internal ListOperation ListOp /// /// Expiration time /// - internal unsafe bool CheckExpiry(long expireTime) - { - if ((flags & RespInputFlags.Deterministic) != 0) - { - if ((flags & RespInputFlags.Expired) != 0) - return true; - } - else - { - if (expireTime < DateTimeOffset.Now.UtcTicks) - { - return true; - } - } - return false; - } + internal readonly unsafe bool CheckExpiry(long expireTime) + => (flags & RespInputFlags.Deterministic) != 0 + ? (flags & RespInputFlags.Expired) != 0 + : expireTime < DateTimeOffset.Now.UtcTicks; /// /// Check the SetGet flag @@ -188,9 +171,9 @@ internal unsafe bool CheckSetGetFlag() => (byte*)Unsafe.AsPointer(ref cmd); /// - /// Get header as SpanByte + /// Get header as PinnedSpanByte /// - public unsafe SpanByte SpanByte => new(Length, (nint)ToPointer()); + public unsafe PinnedSpanByte SpanByte => PinnedSpanByte.FromPinnedPointer(ToPointer(), Length); /// /// Get header length @@ -276,7 +259,7 @@ public unsafe int CopyTo(byte* dest, int length) var curr = dest; // Serialize header - header.SpanByte.CopyTo(curr); + header.SpanByte.SerializeTo(curr); curr += header.SpanByte.TotalSize; // Serialize arg1 @@ -289,7 +272,7 @@ public unsafe int CopyTo(byte* dest, int length) // Serialize parse state var remainingLength = length - (int)(curr - dest); - var len = parseState.CopyTo(curr, remainingLength); + var len = parseState.SerializeTo(curr, remainingLength); curr += len; // Number of serialized bytes @@ -302,10 +285,10 @@ public unsafe int DeserializeFrom(byte* src) var curr = src; // Deserialize header - ref var sbHeader = ref Unsafe.AsRef(curr); - ref var h = ref Unsafe.AsRef(sbHeader.ToPointer()); - curr += sbHeader.TotalSize; - header = h; + var header = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + ref var h = ref Unsafe.AsRef(header.ToPointer()); + curr += header.TotalSize; + this.header = h; // Deserialize arg1 arg1 = *(int*)curr; @@ -405,7 +388,7 @@ public unsafe int CopyTo(byte* dest, int length) var curr = dest; // Serialize header - header.SpanByte.CopyTo(curr); + header.SpanByte.SerializeTo(curr); curr += header.SpanByte.TotalSize; // Serialize arg1 @@ -414,7 +397,7 @@ public unsafe int CopyTo(byte* dest, int length) // Serialize parse state var remainingLength = length - (int)(curr - dest); - var len = parseState.CopyTo(curr, remainingLength); + var len = parseState.SerializeTo(curr, remainingLength); curr += len; // Serialize length @@ -427,10 +410,10 @@ public unsafe int DeserializeFrom(byte* src) var curr = src; // Deserialize header - ref var sbHeader = ref Unsafe.AsRef(curr); - ref var h = ref Unsafe.AsRef(sbHeader.ToPointer()); - curr += sbHeader.TotalSize; - header = h; + var header = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + ref var h = ref Unsafe.AsRef(header.ToPointer()); + curr += header.TotalSize; + this.header = h; // Deserialize arg1 arg1 = *(long*)curr; @@ -496,7 +479,7 @@ public unsafe int CopyTo(byte* dest, int length) // Serialize parse state var remainingLength = (int)(curr - dest); - var len = parseState.CopyTo(curr, remainingLength); + var len = parseState.SerializeTo(curr, remainingLength); curr += len; return (int)(curr - dest); diff --git a/libs/server/Lua/LuaCommands.cs b/libs/server/Lua/LuaCommands.cs index cfe941db52f..cf88de98747 100644 --- a/libs/server/Lua/LuaCommands.cs +++ b/libs/server/Lua/LuaCommands.cs @@ -39,7 +39,7 @@ private unsafe bool TryEVALSHA() ScriptHashKey scriptKey = default; // Length check is mandatory, as ScriptHashKey assumes correct length - if (digest.length == SessionScriptCache.SHA1Len) + if (digest.Length == SessionScriptCache.SHA1Len) { tryAgain: scriptKey = new ScriptHashKey(digest.Span); @@ -174,7 +174,7 @@ private bool NetworkScriptExists() var exists = 0; // Length check is required, as ScriptHashKey makes a hard assumption - if (sha1.length == SessionScriptCache.SHA1Len) + if (sha1.Length == SessionScriptCache.SHA1Len) { AsciiUtils.ToLowerInPlace(sha1.Span); diff --git a/libs/server/Lua/LuaRunner.Functions.cs b/libs/server/Lua/LuaRunner.Functions.cs index 0aa8c5d4e2b..327a4cfc526 100644 --- a/libs/server/Lua/LuaRunner.Functions.cs +++ b/libs/server/Lua/LuaRunner.Functions.cs @@ -15,6 +15,7 @@ using Garnet.common; using KeraLua; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.server { @@ -226,7 +227,7 @@ internal int UnsafeRunPreambleForSession(nint luaStatePtr) if (txnMode) { txnKeyEntries.AddKey(key, false, Tsavorite.core.LockType.Exclusive); - if (!respServerSession.storageSession.objectStoreLockableContext.IsNull) + if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) txnKeyEntries.AddKey(key, true, Tsavorite.core.LockType.Exclusive); } @@ -316,7 +317,7 @@ internal int NoSessionResponse(nint luaStatePtr) /// Entry point for redis.call method from a Lua script (transactional mode) /// public int GarnetCallWithTransaction(nint luaStatePtr) - => ProcessCommandFromScripting(luaStatePtr, ref respServerSession.lockableGarnetApi); + => ProcessCommandFromScripting(luaStatePtr, ref respServerSession.transactionalGarnetApi); /// /// Entry point for redis.call method from a Lua script (non-transactional mode) @@ -3207,8 +3208,8 @@ private unsafe int ProcessCommandFromScripting(nint luaStatePtr, ref } // Note these spans are implicitly pinned, as they're actually on the Lua stack - var key = ArgSlice.FromPinnedSpan(keySpan); - var value = ArgSlice.FromPinnedSpan(valSpan); + var key = PinnedSpanByte.FromPinnedSpan(keySpan); + var value = PinnedSpanByte.FromPinnedSpan(valSpan); _ = api.SET(key, value); @@ -3241,8 +3242,8 @@ private unsafe int ProcessCommandFromScripting(nint luaStatePtr, ref } // Span is (implicitly) pinned since it's actually on the Lua stack - var key = ArgSlice.FromPinnedSpan(keySpan); - var status = api.GET(key, out var value); + var key = PinnedSpanByte.FromPinnedSpan(keySpan); + var status = api.GET(key, out PinnedSpanByte value); if (status == GarnetStatus.OK) { diff --git a/libs/server/Lua/LuaRunner.cs b/libs/server/Lua/LuaRunner.cs index 2d1494f0c81..3f2c0697574 100644 --- a/libs/server/Lua/LuaRunner.cs +++ b/libs/server/Lua/LuaRunner.cs @@ -263,7 +263,7 @@ public unsafe LuaRunner( delegate* unmanaged[Cdecl] garnetCall; if (txnMode) { - txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.lockableContext, respServerSession.storageSession.objectStoreLockableContext); + txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.transactionalContext, respServerSession.storageSession.objectStoreTransactionalContext); garnetCall = &LuaRunnerTrampolines.GarnetCallWithTransaction; } @@ -1235,7 +1235,7 @@ public unsafe object RunForRunner(string[] keys = null, string[] argv = null) { var _key = scratchBufferManager.CreateArgSlice(key); txnKeyEntries.AddKey(_key, false, Tsavorite.core.LockType.Exclusive); - if (!respServerSession.storageSession.objectStoreLockableContext.IsNull) + if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) txnKeyEntries.AddKey(_key, true, Tsavorite.core.LockType.Exclusive); } @@ -1334,25 +1334,25 @@ private void RunInTransaction(ref TResponse response) var txnVersion = respServerSession.storageSession.stateMachineDriver.AcquireTransactionVersion(); try { - respServerSession.storageSession.lockableContext.BeginLockable(); - if (!respServerSession.storageSession.objectStoreLockableContext.IsNull) - respServerSession.storageSession.objectStoreLockableContext.BeginLockable(); + respServerSession.storageSession.transactionalContext.BeginTransaction(); + if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) + respServerSession.storageSession.objectStoreTransactionalContext.BeginTransaction(); respServerSession.SetTransactionMode(true); txnKeyEntries.LockAllKeys(); txnVersion = respServerSession.storageSession.stateMachineDriver.VerifyTransactionVersion(txnVersion); - respServerSession.storageSession.lockableContext.LocksAcquired(txnVersion); - if (!respServerSession.storageSession.objectStoreLockableContext.IsNull) - respServerSession.storageSession.objectStoreLockableContext.LocksAcquired(txnVersion); + respServerSession.storageSession.transactionalContext.LocksAcquired(txnVersion); + if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) + respServerSession.storageSession.objectStoreTransactionalContext.LocksAcquired(txnVersion); RunCommon(ref response); } finally { txnKeyEntries.UnlockAllKeys(); respServerSession.SetTransactionMode(false); - respServerSession.storageSession.lockableContext.EndLockable(); - if (!respServerSession.storageSession.objectStoreLockableContext.IsNull) - respServerSession.storageSession.objectStoreLockableContext.EndLockable(); + respServerSession.storageSession.transactionalContext.EndTransaction(); + if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull) + respServerSession.storageSession.objectStoreTransactionalContext.EndTransaction(); respServerSession.storageSession.stateMachineDriver.EndTransaction(txnVersion); } } diff --git a/libs/server/Lua/ScratchBufferNetworkSender.cs b/libs/server/Lua/ScratchBufferNetworkSender.cs index 9a56a343270..def06b21dec 100644 --- a/libs/server/Lua/ScratchBufferNetworkSender.cs +++ b/libs/server/Lua/ScratchBufferNetworkSender.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using Garnet.networking; +using Tsavorite.core; namespace Garnet.server { @@ -32,7 +33,7 @@ public ScratchBufferNetworkSender() scratchBufferManager = new(); } - public ArgSlice GetResponse() + public PinnedSpanByte GetResponse() => scratchBufferManager.ViewFullArgSlice(); public void Reset() diff --git a/libs/server/Metrics/Info/GarnetInfoMetrics.cs b/libs/server/Metrics/Info/GarnetInfoMetrics.cs index 69098f1aabe..21b3932b4ac 100644 --- a/libs/server/Metrics/Info/GarnetInfoMetrics.cs +++ b/libs/server/Metrics/Info/GarnetInfoMetrics.cs @@ -240,7 +240,6 @@ private MetricsItem[] GetDatabaseStoreStats(StoreWrapper storeWrapper, GarnetDat 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.FixedRecordSize", db.MainStore.Log.FixedRecordSize.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()), @@ -275,7 +274,6 @@ private MetricsItem[] GetDatabaseObjectStoreStats(StoreWrapper storeWrapper, Gar 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.FixedRecordSize", db.ObjectStore.Log.FixedRecordSize.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()), diff --git a/libs/server/Metrics/Slowlog/RespSlowlogCommands.cs b/libs/server/Metrics/Slowlog/RespSlowlogCommands.cs index db022d291a4..d0a599dae2a 100644 --- a/libs/server/Metrics/Slowlog/RespSlowlogCommands.cs +++ b/libs/server/Metrics/Slowlog/RespSlowlogCommands.cs @@ -168,7 +168,7 @@ void HandleSlowLog(RespCommand cmd) byte[] args = new byte[len]; fixed (byte* argsPtr = args) { - parseState.CopyTo(argsPtr, len); + parseState.SerializeTo(argsPtr, len); } entry.Arguments = args; } diff --git a/libs/server/Objects/Hash/HashObject.cs b/libs/server/Objects/Hash/HashObject.cs index 86032323197..4b720b363b7 100644 --- a/libs/server/Objects/Hash/HashObject.cs +++ b/libs/server/Objects/Hash/HashObject.cs @@ -57,8 +57,8 @@ public partial class HashObject : GarnetObjectBase /// /// Constructor /// - public HashObject(long expiration = 0) - : base(expiration, MemoryUtils.DictionaryOverhead) + public HashObject() + : base(new(MemoryUtils.DictionaryOverhead, sizeof(int))) { hash = new Dictionary(ByteArrayComparer.Instance); } @@ -67,7 +67,7 @@ public HashObject(long expiration = 0) /// Construct from binary serialized form /// public HashObject(BinaryReader reader) - : base(reader, MemoryUtils.DictionaryOverhead) + : base(reader, new(MemoryUtils.DictionaryOverhead, sizeof(int))) { hash = new Dictionary(ByteArrayComparer.Instance); @@ -90,7 +90,7 @@ public HashObject(BinaryReader reader) InitializeExpirationStructures(); expirationTimes.Add(item, expiration); expirationQueue.Enqueue(item, expiration); - UpdateExpirationSize(item, true); + UpdateExpirationSize(add: true); } } else @@ -98,15 +98,16 @@ public HashObject(BinaryReader reader) hash.Add(item, value); } - this.UpdateSize(item, value); + // Expiration has already been added via UpdateExpirationSize if hasExpiration + this.UpdateSize(item, value, add: true); } } /// /// Copy constructor /// - public HashObject(Dictionary hash, Dictionary expirationTimes, PriorityQueue expirationQueue, long expiration, long size) - : base(expiration, size) + public HashObject(Dictionary hash, Dictionary expirationTimes, PriorityQueue expirationQueue, ObjectSizes sizes) + : base(sizes) { this.hash = hash; this.expirationTimes = expirationTimes; @@ -152,13 +153,13 @@ public override void DoSerialize(BinaryWriter writer) public override void Dispose() { } /// - public override GarnetObjectBase Clone() => new HashObject(hash, expirationTimes, expirationQueue, Expiration, Size); + public override GarnetObjectBase Clone() => new HashObject(hash, expirationTimes, expirationQueue, sizes); /// public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, - byte respProtocolVersion, out long sizeChange) + byte respProtocolVersion, out long memorySizeChange) { - sizeChange = 0; + memorySizeChange = 0; if (input.header.type != GarnetObjectType.Hash) { @@ -168,7 +169,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var previousSize = this.Size; + var previousMemorySize = this.MemorySize; switch (input.header.HashOp) { case HashOperation.HSET: @@ -235,7 +236,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {input.header.HashOp} in HashObject.Operate"); } - sizeChange = this.Size - previousSize; + memorySizeChange = this.MemorySize - previousMemorySize; if (hash.Count == 0) output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; @@ -243,12 +244,26 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - private void UpdateSize(ReadOnlySpan key, ReadOnlySpan value, bool add = true) + private void UpdateSize(ReadOnlySpan key, ReadOnlySpan value, bool add) { - var size = Utility.RoundUp(key.Length, IntPtr.Size) + Utility.RoundUp(value.Length, IntPtr.Size) + var memorySize = Utility.RoundUp(key.Length, IntPtr.Size) + Utility.RoundUp(value.Length, IntPtr.Size) + (2 * MemoryUtils.ByteArrayOverhead) + MemoryUtils.DictionaryEntryOverhead; - this.Size += add ? size : -size; - Debug.Assert(this.Size >= MemoryUtils.DictionaryOverhead); + + // Expired items are removed before this is called or we're called after UpdateExpirationSize, so we don't adjust for expiration + var kvSize = sizeof(int) * 2 + key.Length + value.Length; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += kvSize; + } + else + { + this.MemorySize -= memorySize; + this.DiskSize -= kvSize; + Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(this.DiskSize >= sizeof(int)); + } } [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -258,26 +273,41 @@ private void InitializeExpirationStructures() { expirationTimes = new Dictionary(ByteArrayComparer.Instance); expirationQueue = new PriorityQueue(); - this.Size += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.MemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + // No DiskSize adjustment needed yet; wait until keys are added or removed } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void UpdateExpirationSize(ReadOnlySpan key, bool add = true) + private void UpdateExpirationSize(bool add, bool includePQ = true) { // Account for dictionary entry and priority queue entry - var size = IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead - + IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; - this.Size += add ? size : -size; + var memorySize = IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + if (includePQ) + memorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += 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)); + } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void CleanupExpirationStructures() + private void CleanupExpirationStructuresIfEmpty() { if (expirationTimes.Count == 0) { - this.Size -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; - this.Size -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.MemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; + this.MemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.DiskSize -= sizeof(long) * expirationTimes.Count; expirationTimes = null; expirationQueue = null; } @@ -317,9 +347,7 @@ public override unsafe void Scan(long start, out List items, out long cu { items.Add(item.Key); if (!isNoValue) - { items.Add(item.Value); - } } else { @@ -329,9 +357,7 @@ public override unsafe void Scan(long start, out List items, out long cu { items.Add(item.Key); if (!isNoValue) - { items.Add(item.Value); - } } } } @@ -340,7 +366,6 @@ public override unsafe void Scan(long start, out List items, out long cu if (items.Count == count) break; - } // Indicates end of collection has been reached. @@ -361,29 +386,33 @@ private void DeleteExpiredItems() private void DeleteExpiredItemsWorker() { + // The PQ is ordered such that oldest items are dequeued first while (expirationQueue.TryPeek(out var key, out var expiration) && expiration < DateTimeOffset.UtcNow.Ticks) { // expirationTimes and expirationQueue will be out of sync when user is updating the expire time of key which already has some TTL. // PriorityQueue Doesn't have update option, so we will just enqueue the new expiration and already treat expirationTimes as the source of truth if (expirationTimes.TryGetValue(key, out var actualExpiration) && actualExpiration == expiration) { - expirationTimes.Remove(key); - expirationQueue.Dequeue(); - UpdateExpirationSize(key, false); + _ = expirationTimes.Remove(key); + _ = expirationQueue.Dequeue(); + UpdateExpirationSize(add: false); if (hash.TryGetValue(key, out var value)) { - hash.Remove(key); - UpdateSize(key, value, false); + _ = hash.Remove(key); + UpdateSize(key, value, add: false); } } else { - expirationQueue.Dequeue(); - this.Size -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); + // The key was not in expirationTimes. It may have been Remove()d. + _ = 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); } } - CleanupExpirationStructures(); + CleanupExpirationStructuresIfEmpty(); } [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -391,9 +420,7 @@ private bool TryGetValue(byte[] key, out byte[] value) { value = default; if (IsExpired(key)) - { return false; - } return hash.TryGetValue(key, out value); } @@ -401,46 +428,42 @@ private bool Remove(byte[] key, out byte[] value) { DeleteExpiredItems(); var result = hash.Remove(key, out value); + if (result) { - UpdateSize(key, value, false); + 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. + _ = expirationTimes.Remove(key); + UpdateExpirationSize(add: false, includePQ: false); + } + UpdateSize(key, value, add: false); } return result; } private int Count() { - if (expirationTimes is null) - { + if (!HasExpirableItems()) return hash.Count; - } var expiredKeysCount = 0; foreach (var item in expirationTimes) { if (IsExpired(item.Key)) - { expiredKeysCount++; - } } - return hash.Count - expiredKeysCount; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool HasExpirableItems() - { - return expirationTimes is not null; - } + private bool HasExpirableItems() => expirationTimes is not null; private bool ContainsKey(byte[] key) { var result = hash.ContainsKey(key); if (result && IsExpired(key)) - { return false; - } - return result; } @@ -449,45 +472,42 @@ private void Add(byte[] key, byte[] value) { DeleteExpiredItems(); hash.Add(key, value); - UpdateSize(key, value); + + // Add() is not called in a context that includes expiration + UpdateSize(key, value, add: true); } private void Set(byte[] key, byte[] value) { - DeleteExpiredItems(); - hash[key] = value; - // Skip overhead as existing item is getting replaced. - this.Size += Utility.RoundUp(value.Length, IntPtr.Size) - - Utility.RoundUp(value.Length, IntPtr.Size); + SetWithoutPersist(key, value); - // To persist the key, if it has an expiration - if (expirationTimes is not null && expirationTimes.TryGetValue(key, out var currentExpiration)) + // Persist the key, if it has an expiration + if (expirationTimes is not null && expirationTimes.Remove(key)) { - expirationTimes.Remove(key); - this.Size -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; - CleanupExpirationStructures(); + this.MemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + this.DiskSize -= sizeof(long); // expiration value size + CleanupExpirationStructuresIfEmpty(); } } + [MethodImpl(MethodImplOptions.AggressiveInlining)] private void SetWithoutPersist(byte[] key, byte[] value) { + // Called only when we have verified the key exists DeleteExpiredItems(); hash[key] = value; - // Skip overhead as existing item is getting replaced. - this.Size += Utility.RoundUp(value.Length, IntPtr.Size) - - Utility.RoundUp(value.Length, IntPtr.Size); + + // Skip MemorySize adjustment as existing item is getting replaced. } private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption) { if (!ContainsKey(key)) - { return (int)ExpireResult.KeyNotFound; - } if (expiration <= DateTimeOffset.UtcNow.Ticks) { - Remove(key, out _); + _ = Remove(key, out _); return (int)ExpireResult.KeyAlreadyExpired; } @@ -504,20 +524,19 @@ private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption expirationTimes[key] = expiration; expirationQueue.Enqueue(key, expiration); - // Size of dictionary entry already accounted for as the key already exists - this.Size += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + + // 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; } else { - if ((expireOption & ExpireOption.XX) == ExpireOption.XX || - (expireOption & ExpireOption.GT) == ExpireOption.GT) - { + if ((expireOption & ExpireOption.XX) == ExpireOption.XX || (expireOption & ExpireOption.GT) == ExpireOption.GT) return (int)ExpireResult.ExpireConditionNotMet; - } expirationTimes[key] = expiration; expirationQueue.Enqueue(key, expiration); - UpdateExpirationSize(key); + UpdateExpirationSize(add: true); } return (int)ExpireResult.ExpireUpdated; @@ -526,15 +545,14 @@ private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption private int Persist(byte[] key) { if (!ContainsKey(key)) - { return -2; - } - if (expirationTimes is not null && expirationTimes.TryGetValue(key, out var currentExpiration)) + if (expirationTimes is not null && expirationTimes.TryGetValue(key, out _)) { expirationTimes.Remove(key); - this.Size -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; - CleanupExpirationStructures(); + this.MemorySize -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + this.DiskSize -= sizeof(long); // expiration value size + CleanupExpirationStructuresIfEmpty(); return 1; } @@ -544,15 +562,9 @@ private int Persist(byte[] key) private long GetExpiration(byte[] key) { if (!ContainsKey(key)) - { return -2; - } - if (expirationTimes is not null && expirationTimes.TryGetValue(key, out var expiration)) - { return expiration; - } - return -1; } @@ -564,14 +576,10 @@ private KeyValuePair ElementAt(int index) foreach (var item in hash) { if (IsExpired(item.Key)) - { continue; - } if (currIndex++ == index) - { return item; - } } throw new ArgumentOutOfRangeException("index is outside the bounds of the source sequence."); diff --git a/libs/server/Objects/Hash/HashObjectImpl.cs b/libs/server/Objects/Hash/HashObjectImpl.cs index e9eb1218bca..4d1d689ca15 100644 --- a/libs/server/Objects/Hash/HashObjectImpl.cs +++ b/libs/server/Objects/Hash/HashObjectImpl.cs @@ -19,7 +19,7 @@ private void HashGet(ref ObjectInput input, ref GarnetObjectStoreOutput output, { using var writer = new RespMemoryWriter(respProtocolVersion, ref output.SpanByteAndMemory); - var key = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = input.parseState.GetArgSliceByRef(0).ToArray(); if (TryGetValue(key, out var hashValue)) { @@ -41,7 +41,7 @@ private void HashMultipleGet(ref ObjectInput input, ref GarnetObjectStoreOutput for (var i = 0; i < input.parseState.Count; i++) { - var key = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + var key = input.parseState.GetArgSliceByRef(i).ToArray(); if (TryGetValue(key, out var hashValue)) { @@ -80,12 +80,9 @@ private void HashDelete(ref ObjectInput input, ref GarnetObjectStoreOutput outpu { for (var i = 0; i < input.parseState.Count; i++) { - var key = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); - - if (Remove(key, out var hashValue)) - { + var key = input.parseState.GetArgSliceByRef(i).ToArray(); + if (Remove(key, out _)) output.Header.result1++; - } } } @@ -96,13 +93,13 @@ private void HashLength(ref GarnetObjectStoreOutput output) private void HashStrLength(ref ObjectInput input, ref GarnetObjectStoreOutput output) { - var key = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = input.parseState.GetArgSliceByRef(0).ToArray(); output.Header.result1 = TryGetValue(key, out var hashValue) ? hashValue.Length : 0; } private void HashExists(ref ObjectInput input, ref GarnetObjectStoreOutput output) { - var field = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var field = input.parseState.GetArgSliceByRef(0).ToArray(); output.Header.result1 = ContainsKey(field) ? 1 : 0; } @@ -180,8 +177,8 @@ private void HashSet(ref ObjectInput input, ref GarnetObjectStoreOutput output) var hop = input.header.HashOp; for (var i = 0; i < input.parseState.Count; i += 2) { - var key = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); - var value = input.parseState.GetArgSliceByRef(i + 1).SpanByte.ToByteArray(); + var key = input.parseState.GetArgSliceByRef(i).ToArray(); + var value = input.parseState.GetArgSliceByRef(i + 1).ToArray(); if (!TryGetValue(key, out var hashValue)) { @@ -242,7 +239,7 @@ private void HashIncrement(ref ObjectInput input, ref GarnetObjectStoreOutput ou // This value is used to indicate partial command execution output.Header.result1 = int.MinValue; - var key = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = input.parseState.GetArgSliceByRef(0).ToArray(); var incrSlice = input.parseState.GetArgSliceByRef(1); var valueExists = TryGetValue(key, out var value); @@ -267,7 +264,7 @@ private void HashIncrement(ref ObjectInput input, ref GarnetObjectStoreOutput ou result += incr; var resultSpan = (Span)stackalloc byte[NumUtils.MaximumFormatInt64Length]; - var success = Utf8Formatter.TryFormat(result, resultSpan, out int bytesWritten, + var success = Utf8Formatter.TryFormat(result, resultSpan, out var bytesWritten, format: default); Debug.Assert(success); @@ -278,7 +275,7 @@ private void HashIncrement(ref ObjectInput input, ref GarnetObjectStoreOutput ou } else { - resultBytes = incrSlice.SpanByte.ToByteArray(); + resultBytes = incrSlice.ToArray(); Add(key, resultBytes); } @@ -309,7 +306,7 @@ private void HashIncrement(ref ObjectInput input, ref GarnetObjectStoreOutput ou } else { - resultBytes = incrSlice.SpanByte.ToByteArray(); + resultBytes = incrSlice.ToArray(); Add(key, resultBytes); } diff --git a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs index ac986bc45b5..f36eebc2121 100644 --- a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs +++ b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs @@ -77,7 +77,7 @@ internal bool TryGetObserver(int sessionId, out CollectionItemObserver observer) /// Additional arguments for command /// Result of operation internal async Task GetCollectionItemAsync(RespCommand command, byte[][] keys, - RespServerSession session, double timeoutInSeconds, ArgSlice[] cmdArgs = null) + RespServerSession session, double timeoutInSeconds, PinnedSpanByte[] cmdArgs = null) { var observer = new CollectionItemObserver(session, command, cmdArgs); return await GetCollectionItemAsync(observer, keys, timeoutInSeconds); @@ -94,7 +94,7 @@ internal async Task GetCollectionItemAsync(RespCommand com /// Additional arguments for command /// Result of operation internal async Task MoveCollectionItemAsync(RespCommand command, byte[] srcKey, - RespServerSession session, double timeoutInSeconds, ArgSlice[] cmdArgs) + RespServerSession session, double timeoutInSeconds, PinnedSpanByte[] cmdArgs) { var observer = new CollectionItemObserver(session, command, cmdArgs); return await GetCollectionItemAsync(observer, [srcKey], timeoutInSeconds); @@ -390,7 +390,7 @@ private static bool TryMoveNextListItem(ListObject srcListObj, ListObject dstLis /// BZPOPMIN and BZPOPMAX share same implementation since Dictionary.First() and Last() /// handle the ordering automatically based on sorted set scores /// - private static unsafe bool TryGetNextSetObjects(byte[] key, SortedSetObject sortedSetObj, int count, RespCommand command, ArgSlice[] cmdArgs, out CollectionItemResult result) + private static unsafe bool TryGetNextSetObjects(byte[] key, SortedSetObject sortedSetObj, int count, RespCommand command, PinnedSpanByte[] cmdArgs, out CollectionItemResult result) { result = default; @@ -405,8 +405,8 @@ private static unsafe bool TryGetNextSetObjects(byte[] key, SortedSetObject sort return true; case RespCommand.BZMPOP: - var lowScoresFirst = *(bool*)cmdArgs[0].ptr; - var popCount = *(int*)cmdArgs[1].ptr; + var lowScoresFirst = *(bool*)cmdArgs[0].ToPointer(); + var popCount = *(int*)cmdArgs[1].ToPointer(); popCount = Math.Min(popCount, count); var scores = new double[popCount]; @@ -427,7 +427,7 @@ private static unsafe bool TryGetNextSetObjects(byte[] key, SortedSetObject sort } } - private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, RespCommand command, ArgSlice[] cmdArgs, out int currCount, out CollectionItemResult result) + private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, RespCommand command, PinnedSpanByte[] cmdArgs, out int currCount, out CollectionItemResult result) { currCount = default; result = default; @@ -440,42 +440,37 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp _ => throw new NotSupportedException() }; - ArgSlice dstKey = default; + PinnedSpanByte dstKey = default; if (command == RespCommand.BLMOVE) - { dstKey = cmdArgs[0]; - } + + var asKey = storageSession.scratchBufferManager.CreateArgSlice(key); // Create a transaction if not currently in a running transaction if (storageSession.txnManager.state != TxnState.Running) { Debug.Assert(storageSession.txnManager.state == TxnState.None); createTransaction = true; - var asKey = storageSession.scratchBufferManager.CreateArgSlice(key); storageSession.txnManager.SaveKeyEntryToLock(asKey, true, LockType.Exclusive); if (command == RespCommand.BLMOVE) - { storageSession.txnManager.SaveKeyEntryToLock(dstKey, true, LockType.Exclusive); - } _ = storageSession.txnManager.Run(true); } - var objectLockableContext = storageSession.txnManager.ObjectStoreLockableContext; + var objectTransactionalContext = storageSession.txnManager.ObjectStoreTransactionalContext; try { // Get the object stored at key - var statusOp = storageSession.GET(key, out var osObject, ref objectLockableContext); + var statusOp = storageSession.GET(asKey, out var osObject, ref objectTransactionalContext); if (statusOp == GarnetStatus.NOTFOUND) return false; IGarnetObject dstObj = null; - byte[] arrDstKey = default; if (command == RespCommand.BLMOVE) { - arrDstKey = dstKey.ToArray(); - var dstStatusOp = storageSession.GET(arrDstKey, out var osDstObject, ref objectLockableContext); + var dstStatusOp = storageSession.GET(dstKey, out var osDstObject, ref objectTransactionalContext); if (dstStatusOp != GarnetStatus.NOTFOUND) dstObj = osDstObject.GarnetObject; } @@ -515,14 +510,13 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp if (isSuccessful && newObj) { - isSuccessful = storageSession.SET(arrDstKey, dstList, ref objectLockableContext) == - GarnetStatus.OK; + isSuccessful = storageSession.SET(dstKey, dstList, ref objectTransactionalContext) == GarnetStatus.OK; } return isSuccessful; case RespCommand.BLMPOP: var popDirection = (OperationDirection)cmdArgs[0].ReadOnlySpan[0]; - var popCount = *(int*)(cmdArgs[1].ptr); + var popCount = *(int*)(cmdArgs[1].ToPointer()); popCount = Math.Min(popCount, listObj.LnkList.Count); var items = new byte[popCount][]; diff --git a/libs/server/Objects/ItemBroker/CollectionItemObserver.cs b/libs/server/Objects/ItemBroker/CollectionItemObserver.cs index f9398758c41..648b2882f5f 100644 --- a/libs/server/Objects/ItemBroker/CollectionItemObserver.cs +++ b/libs/server/Objects/ItemBroker/CollectionItemObserver.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System.Threading; +using Tsavorite.core; namespace Garnet.server { @@ -23,7 +24,7 @@ internal class CollectionItemObserver /// /// Additional arguments for the command /// - internal ArgSlice[] CommandArgs { get; } + internal PinnedSpanByte[] CommandArgs { get; } /// /// Status of the observer @@ -50,7 +51,7 @@ internal class CollectionItemObserver /// internal CancellationTokenSource CancellationTokenSource { get; } = new(); - internal CollectionItemObserver(RespServerSession session, RespCommand command, ArgSlice[] commandArgs = null) + internal CollectionItemObserver(RespServerSession session, RespCommand command, PinnedSpanByte[] commandArgs = null) { Session = session; Command = command; diff --git a/libs/server/Objects/List/ListObject.cs b/libs/server/Objects/List/ListObject.cs index ec174a0e3ba..fd8148c1016 100644 --- a/libs/server/Objects/List/ListObject.cs +++ b/libs/server/Objects/List/ListObject.cs @@ -64,8 +64,8 @@ public partial class ListObject : GarnetObjectBase /// /// Constructor /// - public ListObject(long expiration = 0) - : base(expiration, MemoryUtils.ListOverhead) + public ListObject() + : base(new(MemoryUtils.ListOverhead, sizeof(int))) { list = new LinkedList(); } @@ -74,7 +74,7 @@ public ListObject(long expiration = 0) /// Construct from binary serialized form /// public ListObject(BinaryReader reader) - : base(reader, MemoryUtils.ListOverhead) + : base(reader, new(MemoryUtils.ListOverhead, sizeof(int))) { list = new LinkedList(); @@ -91,8 +91,8 @@ public ListObject(BinaryReader reader) /// /// Copy constructor /// - public ListObject(LinkedList list, long expiration, long size) - : base(expiration, size) + public ListObject(LinkedList list, ObjectSizes sizes) + : base(sizes) { this.list = list; } @@ -125,13 +125,13 @@ public override void DoSerialize(BinaryWriter writer) public override void Dispose() { } /// - public override GarnetObjectBase Clone() => new ListObject(list, Expiration, Size); + public override GarnetObjectBase Clone() => new ListObject(list, sizes); /// public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, - byte respProtocolVersion, out long sizeChange) + byte respProtocolVersion, out long memorySizeChange) { - sizeChange = 0; + memorySizeChange = 0; if (input.header.type != GarnetObjectType.List) { @@ -141,7 +141,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var previousSize = this.Size; + var previousMemorySize = this.MemorySize; 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"); } - sizeChange = this.Size - previousSize; + memorySizeChange = this.MemorySize - previousMemorySize; if (list.Count == 0) output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; @@ -197,9 +197,21 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput internal void UpdateSize(byte[] item, bool add = true) { - var size = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + MemoryUtils.ListEntryOverhead; - this.Size += add ? size : -size; - Debug.Assert(this.Size >= MemoryUtils.ListOverhead); + var memorySize = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + MemoryUtils.ListEntryOverhead; + var diskSize = sizeof(int) + item.Length; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += diskSize; + } + else + { + this.MemorySize -= memorySize; + this.DiskSize -= diskSize; + Debug.Assert(this.MemorySize >= MemoryUtils.ListOverhead); + Debug.Assert(this.DiskSize >= sizeof(int)); + } } /// diff --git a/libs/server/Objects/List/ListObjectImpl.cs b/libs/server/Objects/List/ListObjectImpl.cs index bf1b8fc6a22..3b2b860f637 100644 --- a/libs/server/Objects/List/ListObjectImpl.cs +++ b/libs/server/Objects/List/ListObjectImpl.cs @@ -81,7 +81,7 @@ private void ListInsert(ref ObjectInput input, ref GarnetObjectStoreOutput outpu var pivot = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; // get the string to INSERT into the list - var item = input.parseState.GetArgSliceByRef(2).SpanByte.ToByteArray(); + var item = input.parseState.GetArgSliceByRef(2).ToArray(); var insertBefore = position.EqualsUpperCaseSpanIgnoringCase(CmdStrings.BEFORE); @@ -229,7 +229,7 @@ private void ListPush(ref ObjectInput input, ref GarnetObjectStoreOutput output, output.Header.result1 = 0; for (var i = 0; i < input.parseState.Count; i++) { - var value = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + var value = input.parseState.GetArgSliceByRef(i).ToArray(); // Add the value to the top of the list if (fAddAtHead) @@ -309,7 +309,7 @@ private void ListSet(ref ObjectInput input, ref GarnetObjectStoreOutput output, } // element - var element = input.parseState.GetArgSliceByRef(1).SpanByte.ToByteArray(); + var element = input.parseState.GetArgSliceByRef(1).ToArray(); var targetNode = index == 0 ? list.First : (index == list.Count - 1 ? list.Last diff --git a/libs/server/Objects/Set/SetObject.cs b/libs/server/Objects/Set/SetObject.cs index 4e9ee621c23..87252a3c8df 100644 --- a/libs/server/Objects/Set/SetObject.cs +++ b/libs/server/Objects/Set/SetObject.cs @@ -50,8 +50,8 @@ public partial class SetObject : GarnetObjectBase /// /// Constructor /// - public SetObject(long expiration = 0) - : base(expiration, MemoryUtils.HashSetOverhead) + public SetObject() + : base(new(MemoryUtils.HashSetOverhead, sizeof(int))) { Set = new HashSet(ByteArrayComparer.Instance); @@ -64,7 +64,7 @@ public SetObject(long expiration = 0) /// Construct from binary serialized form /// public SetObject(BinaryReader reader) - : base(reader, MemoryUtils.HashSetOverhead) + : base(reader, new(MemoryUtils.HashSetOverhead, sizeof(int))) { int count = reader.ReadInt32(); @@ -85,8 +85,8 @@ public SetObject(BinaryReader reader) /// /// Copy constructor /// - public SetObject(HashSet set, long expiration, long size) - : base(expiration, size) + public SetObject(HashSet set, ObjectSizes sizes) + : base(sizes) { Set = set; @@ -119,13 +119,13 @@ public override void DoSerialize(BinaryWriter writer) public override void Dispose() { } /// - public override GarnetObjectBase Clone() => new SetObject(Set, Expiration, Size); + public override GarnetObjectBase Clone() => new SetObject(Set, sizes); /// public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, - byte respProtocolVersion, out long sizeChange) + byte respProtocolVersion, out long memorySizeChange) { - sizeChange = 0; + memorySizeChange = 0; if (input.header.type != GarnetObjectType.Set) { @@ -135,7 +135,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var prevSize = this.Size; + var prevMemorySize = this.MemorySize; 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"); } - sizeChange = this.Size - prevSize; + memorySizeChange = this.MemorySize - prevMemorySize; if (Set.Count == 0) output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; @@ -179,9 +179,21 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput internal void UpdateSize(ReadOnlySpan item, bool add = true) { - var size = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + MemoryUtils.HashSetEntryOverhead; - this.Size += add ? size : -size; - Debug.Assert(this.Size >= MemoryUtils.HashSetOverhead); + var memorySize = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + MemoryUtils.HashSetEntryOverhead; + var kvSize = sizeof(int) + item.Length; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += kvSize; + } + else + { + this.MemorySize -= memorySize; + this.DiskSize -= kvSize; + Debug.Assert(this.MemorySize >= MemoryUtils.HashSetOverhead); + Debug.Assert(this.DiskSize >= sizeof(int)); + } } /// diff --git a/libs/server/Objects/SortedSet/SortedSetObject.cs b/libs/server/Objects/SortedSet/SortedSetObject.cs index 164d53cea3a..b008c353c44 100644 --- a/libs/server/Objects/SortedSet/SortedSetObject.cs +++ b/libs/server/Objects/SortedSet/SortedSetObject.cs @@ -177,8 +177,8 @@ public partial class SortedSetObject : GarnetObjectBase /// /// Constructor /// - public SortedSetObject(long expiration = 0) - : base(expiration, MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead) + public SortedSetObject() + : base(new (MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int))) { sortedSet = new(SortedSetComparer.Instance); sortedSetDict = new Dictionary(ByteArrayComparer.Instance); @@ -188,7 +188,7 @@ public SortedSetObject(long expiration = 0) /// Construct from binary serialized form /// public SortedSetObject(BinaryReader reader) - : base(reader, MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead) + : base(reader, new(MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead, sizeof(int))) { sortedSet = new(SortedSetComparer.Instance); sortedSetDict = new Dictionary(ByteArrayComparer.Instance); @@ -221,7 +221,7 @@ public SortedSetObject(BinaryReader reader) InitializeExpirationStructures(); expirationTimes.Add(item, expiration); expirationQueue.Enqueue(item, expiration); - UpdateExpirationSize(item, true); + UpdateExpirationSize(add: true); } } } @@ -231,7 +231,7 @@ public SortedSetObject(BinaryReader reader) /// Copy constructor /// public SortedSetObject(SortedSetObject sortedSetObject) - : base(sortedSetObject.Expiration, sortedSetObject.Size) + : base(sortedSetObject.sizes) { this.sortedSet = sortedSetObject.sortedSet; this.sortedSetDict = sortedSetObject.sortedSetDict; @@ -346,9 +346,9 @@ public override void Dispose() { } /// public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput output, - byte respProtocolVersion, out long sizeChange) + byte respProtocolVersion, out long memorySizeChange) { - sizeChange = 0; + memorySizeChange = 0; var header = input.header; if (header.type != GarnetObjectType.SortedSet) @@ -359,7 +359,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput return true; } - var prevSize = this.Size; + var prevMemorySize = this.MemorySize; var op = header.SortedSetOp; switch (op) { @@ -445,7 +445,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput throw new GarnetException($"Unsupported operation {op} in SortedSetObject.Operate"); } - sizeChange = this.Size - prevSize; + memorySizeChange = this.MemorySize - prevMemorySize; if (sortedSetDict.Count == 0) output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey; @@ -592,9 +592,7 @@ public bool TryGetScore(byte[] key, out double value) { value = default; if (IsExpired(key)) - { return false; - } return sortedSetDict.TryGetValue(key, out value); } @@ -606,17 +604,13 @@ public bool TryGetScore(byte[] key, out double value) public int Count() { if (!HasExpirableItems()) - { return sortedSetDict.Count; - } - var expiredKeysCount = 0; + var expiredKeysCount = 0; foreach (var item in expirationTimes) { if (IsExpired(item.Key)) - { expiredKeysCount++; - } } return sortedSetDict.Count - expiredKeysCount; } @@ -634,10 +628,7 @@ public int Count() /// /// True if the sorted set has expirable items; otherwise, false. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool HasExpirableItems() - { - return expirationTimes is not null; - } + public bool HasExpirableItems() => expirationTimes is not null; #endregion private void InitializeExpirationStructures() @@ -646,25 +637,41 @@ private void InitializeExpirationStructures() { expirationTimes = new Dictionary(ByteArrayComparer.Instance); expirationQueue = new PriorityQueue(); - this.Size += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.MemorySize += MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + // No DiskSize adjustment needed yet; wait until keys are added or removed } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void UpdateExpirationSize(ReadOnlySpan key, bool add = true) + private void UpdateExpirationSize(bool add, bool includePQ = true) { - var size = IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead - + IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; - this.Size += add ? size : -size; + // Account for dictionary entry and priority queue entry + var memorySize = IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; + if (includePQ) + memorySize += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += 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)); + } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void CleanupExpirationStructures() + private void CleanupExpirationStructuresIfEmpty() { if (expirationTimes.Count == 0) { - this.Size -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; - this.Size -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.MemorySize -= (IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueOverhead) * expirationQueue.Count; + this.MemorySize -= MemoryUtils.DictionaryOverhead + MemoryUtils.PriorityQueueOverhead; + this.DiskSize -= sizeof(long) * expirationTimes.Count; expirationTimes = null; expirationQueue = null; } @@ -686,36 +693,37 @@ private void DeleteExpiredItemsWorker() { expirationTimes.Remove(key); expirationQueue.Dequeue(); - UpdateExpirationSize(key, false); + UpdateExpirationSize(add: false); if (sortedSetDict.TryGetValue(key, out var value)) { sortedSetDict.Remove(key); sortedSet.Remove((value, key)); - UpdateSize(key, false); + UpdateSize(key, add: false); } } else { - expirationQueue.Dequeue(); - this.Size -= MemoryUtils.PriorityQueueEntryOverhead + IntPtr.Size + sizeof(long); + // The key was not in expirationTimes. It may have been Remove()d. + _ = 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); } } - CleanupExpirationStructures(); + CleanupExpirationStructuresIfEmpty(); } private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption) { if (!sortedSetDict.ContainsKey(key)) - { return (int)SortedSetExpireResult.KeyNotFound; - } if (expiration <= DateTimeOffset.UtcNow.Ticks) { sortedSetDict.Remove(key, out var value); sortedSet.Remove((value, key)); - UpdateSize(key, false); + UpdateSize(key, add: false); return (int)SortedSetExpireResult.KeyAlreadyExpired; } @@ -732,18 +740,19 @@ private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption expirationTimes[key] = expiration; expirationQueue.Enqueue(key, expiration); - this.Size += IntPtr.Size + sizeof(long) + MemoryUtils.PriorityQueueEntryOverhead; + + // 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; } else { - if (expireOption.HasFlag(ExpireOption.XX) || expireOption.HasFlag(ExpireOption.GT)) - { + if ((expireOption & ExpireOption.XX) == ExpireOption.XX || (expireOption & ExpireOption.GT) == ExpireOption.GT) return (int)SortedSetExpireResult.ExpireConditionNotMet; - } expirationTimes[key] = expiration; expirationQueue.Enqueue(key, expiration); - UpdateExpirationSize(key); + UpdateExpirationSize(add: true); } return (int)SortedSetExpireResult.ExpireUpdated; @@ -752,10 +761,7 @@ private int SetExpiration(byte[] key, long expiration, ExpireOption expireOption private int Persist(byte[] key) { if (!sortedSetDict.ContainsKey(key)) - { return -2; - } - return TryRemoveExpiration(key) ? 1 : -1; } @@ -770,28 +776,21 @@ private bool TryRemoveExpiration(byte[] key) private bool TryRemoveExpirationWorker(byte[] key) { if (!expirationTimes.TryGetValue(key, out _)) - { return false; - } expirationTimes.Remove(key); - this.Size -= IntPtr.Size + sizeof(long) + MemoryUtils.DictionaryEntryOverhead; - CleanupExpirationStructures(); + + UpdateExpirationSize(add: false, includePQ: false); + CleanupExpirationStructuresIfEmpty(); return true; } private long GetExpiration(byte[] key) { if (!sortedSetDict.ContainsKey(key)) - { return -2; - } - if (expirationTimes is not null && expirationTimes.TryGetValue(key, out var expiration)) - { return expiration; - } - return -1; } @@ -803,14 +802,9 @@ private KeyValuePair ElementAt(int index) foreach (var item in sortedSetDict) { if (IsExpired(item.Key)) - { continue; - } - if (currIndex++ == index) - { return item; - } } throw new ArgumentOutOfRangeException("index is outside the bounds of the source sequence."); @@ -822,10 +816,22 @@ private KeyValuePair ElementAt(int index) private void UpdateSize(ReadOnlySpan item, bool add = true) { // item's length + overhead to store item + value of type double added to sorted set and dictionary + overhead for those datastructures - var size = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + (2 * sizeof(double)) + var memorySize = Utility.RoundUp(item.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + (2 * sizeof(double)) + MemoryUtils.SortedSetEntryOverhead + MemoryUtils.DictionaryEntryOverhead; - this.Size += add ? size : -size; - Debug.Assert(this.Size >= MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead); + var kvSize = sizeof(int) + item.Length + sizeof(double); + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += kvSize; + } + else + { + this.MemorySize -= memorySize; + this.DiskSize -= kvSize; + Debug.Assert(this.MemorySize >= MemoryUtils.SortedSetOverhead + MemoryUtils.DictionaryOverhead); + Debug.Assert(this.DiskSize >= sizeof(int)); + } } /// diff --git a/libs/server/Objects/SortedSet/SortedSetObjectImpl.cs b/libs/server/Objects/SortedSet/SortedSetObjectImpl.cs index df928eded45..ee90c230405 100644 --- a/libs/server/Objects/SortedSet/SortedSetObjectImpl.cs +++ b/libs/server/Objects/SortedSet/SortedSetObjectImpl.cs @@ -218,7 +218,7 @@ private void SortedSetScore(ref ObjectInput input, ref GarnetObjectStoreOutput o // ZSCORE key member using var writer = new RespMemoryWriter(respProtocolVersion, ref output.SpanByteAndMemory); - var member = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(0).ToArray(); if (!TryGetScore(member, out var score)) { @@ -242,7 +242,7 @@ private void SortedSetScores(ref ObjectInput input, ref GarnetObjectStoreOutput for (var i = 0; i < count; i++) { - var member = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(i).ToArray(); if (!TryGetScore(member, out var score)) { @@ -279,9 +279,12 @@ private void SortedSetCount(ref ObjectInput input, ref GarnetObjectStoreOutput o { foreach (var item in sortedSet.GetViewBetween((minValue, null), sortedSet.Max)) { - if (IsExpired(item.Element)) continue; - if (item.Score > maxValue || (maxExclusive && item.Score == maxValue)) break; - if (minExclusive && item.Score == minValue) continue; + if (IsExpired(item.Element)) + continue; + if (item.Score > maxValue || (maxExclusive && item.Score == maxValue)) + break; + if (minExclusive && item.Score == minValue) + continue; count++; } } @@ -304,7 +307,7 @@ private void SortedSetIncrement(ref ObjectInput input, ref GarnetObjectStoreOutp } // Read member - var member = input.parseState.GetArgSliceByRef(1).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(1).ToArray(); if (sortedSetDict.TryGetValue(member, out var score)) { @@ -663,7 +666,7 @@ private void SortedSetRank(ref ObjectInput input, ref GarnetObjectStoreOutput ou //ZRANK key member var withScore = input.arg1 == 1; - var member = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(0).ToArray(); using var writer = new RespMemoryWriter(respProtocolVersion, ref output.SpanByteAndMemory); diff --git a/libs/server/Objects/SortedSetGeo/SortedSetGeoObjectImpl.cs b/libs/server/Objects/SortedSetGeo/SortedSetGeoObjectImpl.cs index 107b97bc4f4..2aee87d2997 100644 --- a/libs/server/Objects/SortedSetGeo/SortedSetGeoObjectImpl.cs +++ b/libs/server/Objects/SortedSetGeo/SortedSetGeoObjectImpl.cs @@ -90,7 +90,7 @@ private void GeoHash(ref ObjectInput input, ref GarnetObjectStoreOutput output, for (var i = 0; i < input.parseState.Count; i++) { // Read member - var member = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(i).ToArray(); if (sortedSetDict.TryGetValue(member, out var value52Int)) { @@ -107,10 +107,10 @@ private void GeoHash(ref ObjectInput input, ref GarnetObjectStoreOutput output, private void GeoDistance(ref ObjectInput input, ref GarnetObjectStoreOutput output, byte respProtocolVersion) { // Read 1st member - var member1 = input.parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var member1 = input.parseState.GetArgSliceByRef(0).ToArray(); // Read 2nd member - var member2 = input.parseState.GetArgSliceByRef(1).SpanByte.ToByteArray(); + var member2 = input.parseState.GetArgSliceByRef(1).ToArray(); // Read units var units = GeoDistanceUnitType.M; @@ -149,7 +149,7 @@ private void GeoPosition(ref ObjectInput input, ref GarnetObjectStoreOutput outp for (var i = 0; i < input.parseState.Count; i++) { // read member - var member = input.parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + var member = input.parseState.GetArgSliceByRef(i).ToArray(); if (sortedSetDict.TryGetValue(member, out var scoreMember1)) { diff --git a/libs/server/Objects/Types/ByteArrayBinaryObjectSerializer.cs b/libs/server/Objects/Types/ByteArrayBinaryObjectSerializer.cs deleted file mode 100644 index a3140797580..00000000000 --- a/libs/server/Objects/Types/ByteArrayBinaryObjectSerializer.cs +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using Tsavorite.core; - -namespace Garnet.server -{ - /// - /// Byte array serializer - /// - public sealed class ByteArrayBinaryObjectSerializer : BinaryObjectSerializer - { - /// - public override void Deserialize(out byte[] obj) => obj = reader.ReadBytes(reader.ReadInt32()); - /// - public override void Serialize(ref byte[] obj) - { - writer.Write(obj.Length); - writer.Write(obj); - } - } -} \ No newline at end of file diff --git a/libs/server/Objects/Types/GarnetObject.cs b/libs/server/Objects/Types/GarnetObject.cs index 85167c197b2..edd0b1bec35 100644 --- a/libs/server/Objects/Types/GarnetObject.cs +++ b/libs/server/Objects/Types/GarnetObject.cs @@ -19,10 +19,10 @@ internal static IGarnetObject Create(GarnetObjectType garnetObjectType, long exp { return garnetObjectType switch { - GarnetObjectType.SortedSet => new SortedSetObject(expiration), - GarnetObjectType.List => new ListObject(expiration), - GarnetObjectType.Hash => new HashObject(expiration), - GarnetObjectType.Set => new SetObject(expiration), + GarnetObjectType.SortedSet => new SortedSetObject(), + GarnetObjectType.List => new ListObject(), + GarnetObjectType.Hash => new HashObject(), + GarnetObjectType.Set => new SetObject(), _ => throw new Exception("Unsupported data type"), }; } @@ -77,6 +77,7 @@ internal static bool NeedToCreate(RespInputHeader header) GarnetObjectType.Expire => false, GarnetObjectType.PExpire => false, GarnetObjectType.Persist => false, + GarnetObjectType.Migrate => false, _ => true, }; } diff --git a/libs/server/Objects/Types/GarnetObjectBase.cs b/libs/server/Objects/Types/GarnetObjectBase.cs index 61e98754722..2e1a115694a 100644 --- a/libs/server/Objects/Types/GarnetObjectBase.cs +++ b/libs/server/Objects/Types/GarnetObjectBase.cs @@ -12,6 +12,24 @@ 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 /// @@ -24,21 +42,23 @@ public abstract class GarnetObjectBase : IGarnetObject public abstract byte Type { get; } /// - public long Expiration { get; set; } + public long MemorySize { get => sizes.Memory; set => sizes.Memory = value; } /// - public long Size { get; set; } + public long DiskSize { get => sizes.Disk; set => sizes.Disk = value; } + + public ObjectSizes sizes; - protected GarnetObjectBase(long expiration, long size) + protected GarnetObjectBase(ObjectSizes sizes) { - Debug.Assert(size >= 0); - this.Expiration = expiration; - this.Size = size; + sizes.Verify(); + this.sizes = sizes; } - protected GarnetObjectBase(BinaryReader reader, long size) - : this(expiration: reader.ReadInt64(), size: size) + protected GarnetObjectBase(BinaryReader reader, ObjectSizes sizes) + : this(sizes) { + // Add anything here that should match DoSerialize() } /// @@ -77,17 +97,16 @@ public void Serialize(BinaryWriter writer) } /// - public void CopyUpdate(ref IGarnetObject oldValue, ref IGarnetObject newValue, bool isInNewVersion) + public IGarnetObject CopyUpdate(bool isInNewVersion, ref RMWInfo rmwInfo) { - newValue = Clone(); - newValue.Expiration = Expiration; + 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) { - oldValue = null; - return; + rmwInfo.ClearSourceValueObject = true; + return newValue; } // Create a serialized version for checkpoint version (v) @@ -101,14 +120,16 @@ public void CopyUpdate(ref IGarnetObject oldValue, ref IGarnetObject newValue, b serialized = ms.ToArray(); serializationState = (int)SerializationPhase.SERIALIZED; - return; + break; } if (serializationState >= (int)SerializationPhase.SERIALIZED) - return; + break; - Thread.Yield(); + _ = Thread.Yield(); } + + return newValue; } /// @@ -129,7 +150,7 @@ public void CopyUpdate(ref IGarnetObject oldValue, ref IGarnetObject newValue, b /// public virtual void DoSerialize(BinaryWriter writer) { - writer.Write(Expiration); + // Add anything here that needs to be in front of the derived object data } private bool MakeTransition(SerializationPhase expectedPhase, SerializationPhase nextPhase) @@ -242,7 +263,7 @@ private static unsafe bool ReadScanInput(ref ObjectInput input, ref SpanByteAndM if (sbParam.EqualsUpperCaseSpanIgnoringCase(CmdStrings.MATCH)) { // Read pattern for keys filter - var sbPattern = input.parseState.GetArgSliceByRef(currTokenIdx++).SpanByte; + var sbPattern = input.parseState.GetArgSliceByRef(currTokenIdx++); pattern = sbPattern.ToPointer(); patternLength = sbPattern.Length; } diff --git a/libs/server/Objects/Types/GarnetObjectSerializer.cs b/libs/server/Objects/Types/GarnetObjectSerializer.cs index 498dec7e289..49855c98473 100644 --- a/libs/server/Objects/Types/GarnetObjectSerializer.cs +++ b/libs/server/Objects/Types/GarnetObjectSerializer.cs @@ -11,7 +11,8 @@ namespace Garnet.server /// /// Serializer for IGarnetObject /// - public sealed class GarnetObjectSerializer : BinaryObjectSerializer + /// Implements for Tsavorite + public sealed class GarnetObjectSerializer : BinaryObjectSerializer, IObjectSerializer { readonly CustomCommandManager customCommandManager; @@ -29,6 +30,12 @@ public override void Deserialize(out IGarnetObject obj) obj = DeserializeInternal(base.reader); } + /// + public void Deserialize(out IHeapObject obj) + { + obj = DeserializeInternal(base.reader); + } + /// Thread-safe version of Deserialize /// /// @@ -64,21 +71,24 @@ private IGarnetObject CustomDeserialize(byte type, BinaryReader binaryReader) } /// - public override void Serialize(ref IGarnetObject obj) => SerializeInternal(base.writer, obj); + public override void Serialize(IGarnetObject obj) => SerializeInternal(base.writer, obj); /// Thread safe version of Serialize. /// /// - public static byte[] Serialize(IGarnetObject obj) + public static void Serialize(IGarnetObject obj, out byte[] bytes) { Debug.Assert(obj != null); using var ms = new MemoryStream(); using var binaryWriter = new BinaryWriter(ms, Encoding.UTF8); SerializeInternal(binaryWriter, obj); - return ms.ToArray(); + bytes = ms.ToArray(); } + /// + public void Serialize(IHeapObject obj) => SerializeInternal(base.writer, (IGarnetObject)obj); + private static void SerializeInternal(BinaryWriter binaryWriter, IGarnetObject obj) { if (obj == null) diff --git a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs index ad525e18818..0547c6042b4 100644 --- a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs +++ b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs @@ -58,22 +58,19 @@ public struct GarnetObjectStoreOutput /// /// True if output flag WrongType is set /// - public bool HasWrongType => (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType; + public readonly bool HasWrongType => (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType; /// /// True if output flag RemoveKey is set /// - public bool HasRemoveKey => (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey; + public readonly bool HasRemoveKey => (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey; - public GarnetObjectStoreOutput() - { - SpanByteAndMemory = new(null); - } + public GarnetObjectStoreOutput() => SpanByteAndMemory = new(null); - public GarnetObjectStoreOutput(SpanByteAndMemory spam) - { - SpanByteAndMemory = spam; - } + public GarnetObjectStoreOutput(SpanByteAndMemory spam) => SpanByteAndMemory = spam; + + public static unsafe GarnetObjectStoreOutput FromPinnedPointer(byte* pointer, int length) + => new(new SpanByteAndMemory() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length) }); public void ConvertToHeap() { diff --git a/libs/server/Objects/Types/GarnetObjectType.cs b/libs/server/Objects/Types/GarnetObjectType.cs index ddbc40f8f4d..e5834206afb 100644 --- a/libs/server/Objects/Types/GarnetObjectType.cs +++ b/libs/server/Objects/Types/GarnetObjectType.cs @@ -33,6 +33,11 @@ public enum GarnetObjectType : byte // Any new special type inserted here should update GarnetObjectTypeExtensions.FirstSpecialObjectType + /// + /// Special type indicating MIGRATE command + /// + Migrate = 0xf7, + /// /// Special type indicating PEXPIRE command /// @@ -78,6 +83,6 @@ public static class GarnetObjectTypeExtensions { internal const GarnetObjectType LastObjectType = GarnetObjectType.Set; - internal const GarnetObjectType FirstSpecialObjectType = GarnetObjectType.PExpire; + internal const GarnetObjectType FirstSpecialObjectType = GarnetObjectType.Migrate; } } \ No newline at end of file diff --git a/libs/server/Objects/Types/IGarnetObject.cs b/libs/server/Objects/Types/IGarnetObject.cs index e698eeb4e78..772db75c0e7 100644 --- a/libs/server/Objects/Types/IGarnetObject.cs +++ b/libs/server/Objects/Types/IGarnetObject.cs @@ -1,32 +1,22 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; using System.Collections.Generic; using System.IO; +using Tsavorite.core; namespace Garnet.server { /// /// Interface representing Garnet object /// - public interface IGarnetObject : IDisposable + public interface IGarnetObject : IHeapObject { /// /// Type of object /// byte Type { get; } - /// - /// Expiration time of object - /// - long Expiration { get; set; } - - /// - /// Total memory size of the object - /// - long Size { get; set; } - /// /// Operator on object /// @@ -42,9 +32,9 @@ public interface IGarnetObject : IDisposable void Serialize(BinaryWriter writer); /// - /// Copy update + /// Create a cloned CopyUpdate of this object /// - void CopyUpdate(ref IGarnetObject oldValue, ref IGarnetObject newValue, bool isInNewVersion); + IGarnetObject CopyUpdate(bool isInNewVersion, ref RMWInfo rmwInfo); /// /// Scan the items of the collection diff --git a/libs/server/Providers/GarnetProvider.cs b/libs/server/Providers/GarnetProvider.cs index 1ccbc6ac248..e8b53202805 100644 --- a/libs/server/Providers/GarnetProvider.cs +++ b/libs/server/Providers/GarnetProvider.cs @@ -8,14 +8,13 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; /// - /// Session provider for Garnet, based on - /// [K, V, I, O, C] = [SpanByte, SpanByte, SpanByte, SpanByteAndMemory, long] + /// Session provider for Garnet /// - public sealed class GarnetProvider : TsavoriteKVProviderBase, MainStoreFunctions, MainStoreAllocator, SpanByteServerSerializer> + public sealed class GarnetProvider : TsavoriteKVProviderBase { readonly StoreWrapper storeWrapper; @@ -32,10 +31,8 @@ public sealed class GarnetProvider : TsavoriteKVProviderBase /// /// - public GarnetProvider(StoreWrapper storeWrapper, - SubscribeBroker broker = null, - MaxSizeSettings maxSizeSettings = default) - : base(new SpanByteServerSerializer(), broker, maxSizeSettings) + public GarnetProvider(StoreWrapper storeWrapper, SubscribeBroker broker = null, MaxSizeSettings maxSizeSettings = default) + : base(broker, maxSizeSettings) { this.storeWrapper = storeWrapper; } @@ -60,13 +57,10 @@ public void Dispose() storeWrapper.Dispose(); } - /// - public override SpanByteFunctionsForServer GetFunctions() => new(); - /// public override IMessageConsumer GetSession(WireFormat wireFormat, INetworkSender networkSender) => (wireFormat == WireFormat.ASCII) - ? new RespServerSession(Interlocked.Increment(ref lastSessionId), networkSender, storeWrapper, broker, null, true) + ? new RespServerSession(Interlocked.Increment(ref lastSessionId), networkSender, storeWrapper, broker, authenticator: null, enableScripts: true) : throw new GarnetException($"Unsupported wireFormat {wireFormat}"); } } \ No newline at end of file diff --git a/libs/server/Providers/TsavoriteKVProviderBase.cs b/libs/server/Providers/TsavoriteKVProviderBase.cs index 85a6e84e7b3..2f805fd8363 100644 --- a/libs/server/Providers/TsavoriteKVProviderBase.cs +++ b/libs/server/Providers/TsavoriteKVProviderBase.cs @@ -7,24 +7,16 @@ namespace Garnet.server { /// - /// Abstract session provider for TsavoriteKV store based on - /// [K, V, I, O, F, P] + /// Abstract session provider for TsavoriteKV store /// - public abstract class TsavoriteKVProviderBase : ISessionProvider - where TSessionFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - where TParameterSerializer : IServerSerializer + public abstract class TsavoriteKVProviderBase : ISessionProvider + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Store /// - protected readonly TsavoriteKV store; - - /// - /// Serializer - /// - protected readonly TParameterSerializer serializer; + protected readonly TsavoriteKV store; /// /// Broker @@ -39,14 +31,11 @@ public abstract class TsavoriteKVProviderBase /// Create TsavoriteKV backend /// - /// /// /// - public TsavoriteKVProviderBase(TParameterSerializer serializer, - SubscribeBroker broker = null, MaxSizeSettings maxSizeSettings = default) + public TsavoriteKVProviderBase(SubscribeBroker broker = null, MaxSizeSettings maxSizeSettings = default) { this.broker = broker; - this.serializer = serializer; this.maxSizeSettings = maxSizeSettings ?? new MaxSizeSettings(); } @@ -55,12 +44,6 @@ public TsavoriteKVProviderBase(TParameterSerializer serializer, /// public MaxSizeSettings GetMaxSizeSettings => this.maxSizeSettings; - /// - /// GetFunctions() for custom functions provided by the client - /// - /// - public abstract TSessionFunctions GetFunctions(); - /// public abstract IMessageConsumer GetSession(WireFormat wireFormat, INetworkSender networkSender); } diff --git a/libs/server/PubSub/SubscribeBroker.cs b/libs/server/PubSub/SubscribeBroker.cs index d55e3623eb8..362a813f233 100644 --- a/libs/server/PubSub/SubscribeBroker.cs +++ b/libs/server/PubSub/SubscribeBroker.cs @@ -22,7 +22,7 @@ public sealed class SubscribeBroker : IDisposable, ILogEntryConsumer bool initialized = false; ConcurrentDictionary> subscriptions; ReadOptimizedConcurrentSet patternSubscriptions; - readonly TsavoriteLog log; + readonly TsavoriteLog aof; readonly IDevice device; readonly CancellationTokenSource cts = new(); readonly ManualResetEvent done = new(true); @@ -42,10 +42,10 @@ public SubscribeBroker(string logDir, long pageSize, int subscriberRefreshFreque { device = logDir == null ? new NullDevice() : Devices.CreateLogDevice(logDir + "/pubsubkv", preallocateFile: false); device.Initialize((long)(1 << 30) * 64); - log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = pageSize * 4, SafeTailRefreshFrequencyMs = subscriberRefreshFrequencyMs }); - pageSizeBits = log.UnsafeGetLogPageSizeBits(); + aof = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = pageSize * 4, SafeTailRefreshFrequencyMs = subscriberRefreshFrequencyMs }); + pageSizeBits = aof.UnsafeGetLogPageSizeBits(); if (startFresh) - log.TruncateUntil(log.CommittedUntilAddress); + aof.TruncateUntil(aof.CommittedUntilAddress); this.logger = logger; } @@ -74,7 +74,7 @@ public unsafe void RemoveSubscription(IMessageConsumer session) } } - unsafe int Broadcast(ArgSlice key, ArgSlice value) + unsafe int Broadcast(PinnedSpanByte key, PinnedSpanByte value) { var numSubscribers = 0; @@ -99,7 +99,7 @@ unsafe int Broadcast(ArgSlice key, ArgSlice value) var pattern = entry.pattern; fixed (byte* patternPtr = pattern.ReadOnlySpan) { - var patternSlice = new ArgSlice(patternPtr, pattern.ReadOnlySpan.Length); + var patternSlice = PinnedSpanByte.FromPinnedPointer(patternPtr, pattern.ReadOnlySpan.Length); if (Match(key, patternSlice)) { var sessions = entry.subscriptions; @@ -120,7 +120,7 @@ async Task Start(CancellationToken cancellationToken = default) { try { - using var iterator = log.ScanSingle(log.BeginAddress, long.MaxValue, scanUncommitted: true); + using var iterator = aof.ScanSingle(aof.BeginAddress, long.MaxValue, scanUncommitted: true); var signal = iterator.Signal; using var registration = cts.Token.Register(signal); @@ -154,12 +154,12 @@ public unsafe void Consume(byte* payloadPtr, int payloadLength, long currentAddr } var ptr = payloadPtr; - var key = new ArgSlice(ptr + sizeof(int), *(int*)ptr); - ptr += sizeof(int) + key.length; - var value = new ArgSlice(ptr + sizeof(int), *(int*)ptr); + var key = PinnedSpanByte.FromPinnedPointer(ptr + sizeof(int), *(int*)ptr); + ptr += sizeof(int) + key.Length; + var value = PinnedSpanByte.FromPinnedPointer(ptr + sizeof(int), *(int*)ptr); _ = Broadcast(key, value); - if (nextAddress > log.BeginAddress) - log.TruncateUntil(nextAddress); + if (nextAddress > aof.BeginAddress) + aof.TruncateUntil(nextAddress); previousAddress = nextAddress; } catch (Exception ex) @@ -184,7 +184,7 @@ void Initialize() /// Key to subscribe to /// Server session /// - public unsafe bool Subscribe(ArgSlice key, ServerSessionBase session) + public unsafe bool Subscribe(PinnedSpanByte key, ServerSessionBase session) { if (!initialized && Interlocked.Increment(ref sid) == 1) Initialize(); @@ -204,7 +204,7 @@ public unsafe bool Subscribe(ArgSlice key, ServerSessionBase session) /// Pattern to subscribe to /// Server session /// - public unsafe bool PatternSubscribe(ArgSlice pattern, ServerSessionBase session) + public unsafe bool PatternSubscribe(PinnedSpanByte pattern, ServerSessionBase session) { if (!initialized && Interlocked.Increment(ref sid) == 1) Initialize(); @@ -295,7 +295,7 @@ public unsafe List ListAllPatternSubscriptions(ServerSessionBa /// key that has been updated /// value that has been updated /// Number of subscribers notified - public unsafe int PublishNow(ArgSlice key, ArgSlice value) + public unsafe int PublishNow(PinnedSpanByte key, PinnedSpanByte value) { if (subscriptions == null && patternSubscriptions == null) return 0; return Broadcast(key, value); @@ -306,13 +306,12 @@ public unsafe int PublishNow(ArgSlice key, ArgSlice value) /// /// key that has been updated /// value that has been updated - public unsafe void Publish(ArgSlice key, ArgSlice value) + public unsafe void Publish(PinnedSpanByte key, PinnedSpanByte value) { - if (subscriptions == null && patternSubscriptions == null) return; + if (subscriptions == null && patternSubscriptions == null) + return; - var keySB = key.SpanByte; - var valueSB = value.SpanByte; - log.Enqueue(ref keySB, ref valueSB, out _); + aof.Enqueue(key.ReadOnlySpan, value.ReadOnlySpan, out _); } /// @@ -338,7 +337,7 @@ public List GetChannels() /// /// /// - public unsafe List GetChannels(ArgSlice pattern) + public unsafe List GetChannels(PinnedSpanByte pattern) { if (subscriptions is null || subscriptions.IsEmpty) return []; @@ -350,7 +349,7 @@ public unsafe List GetChannels(ArgSlice pattern) { fixed (byte* keyPtr = entry.Key.ReadOnlySpan) { - if (Match(new ArgSlice(keyPtr, entry.Key.ReadOnlySpan.Length), pattern)) + if (Match(PinnedSpanByte.FromPinnedPointer(keyPtr, entry.Key.ReadOnlySpan.Length), pattern)) channels.Add(entry.Key); } } @@ -382,7 +381,7 @@ public int NumPatternSubscriptions() /// /// /// - public int NumSubscriptions(ArgSlice channel) + public int NumSubscriptions(PinnedSpanByte channel) { if (subscriptions is null) return 0; @@ -398,11 +397,11 @@ public void Dispose() done.WaitOne(); subscriptions?.Clear(); patternSubscriptions?.Clear(); - log.Dispose(); + aof.Dispose(); device.Dispose(); } - unsafe bool Match(ArgSlice key, ArgSlice pattern) - => GlobUtils.Match(pattern.ptr, pattern.length, key.ptr, key.length); + unsafe bool Match(PinnedSpanByte key, PinnedSpanByte pattern) + => GlobUtils.Match(pattern.ToPointer(), pattern.Length, key.ToPointer(), key.Length); } } \ No newline at end of file diff --git a/libs/server/Resp/ArrayCommands.cs b/libs/server/Resp/ArrayCommands.cs index 12d5eb5e381..e6e506fe063 100644 --- a/libs/server/Resp/ArrayCommands.cs +++ b/libs/server/Resp/ArrayCommands.cs @@ -30,9 +30,9 @@ private bool NetworkMGET(ref TGarnetApi storageApi) for (var c = 0; c < parseState.Count; c++) { - var key = parseState.GetArgSliceByRef(c).SpanByte; - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.GET(ref key, ref input, ref o); + var key = parseState.GetArgSliceByRef(c); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.GET(key, ref input, ref o); switch (status) { @@ -65,16 +65,16 @@ private bool NetworkMGET_SG(ref TGarnetApi storageApi) SendAndReset(); RawStringInput input = default; - SpanByteAndMemory o = new(dcurr, (int)(dend - dcurr)); + SpanByteAndMemory o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); for (var c = 0; c < parseState.Count; c++) { - var key = parseState.GetArgSliceByRef(c).SpanByte; + var key = parseState.GetArgSliceByRef(c); // Store index in context, since completions are not in order long ctx = c; - var status = storageApi.GET_WithPending(ref key, ref input, ref o, ctx, out var isPending); + var status = storageApi.GET_WithPending(key, ref input, ref o, ctx, out var isPending); if (isPending) { @@ -97,7 +97,7 @@ private bool NetworkMGET_SG(ref TGarnetApi storageApi) SendAndReset(o.Memory, o.Length); else dcurr += o.Length; - o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); } else { @@ -111,7 +111,7 @@ private bool NetworkMGET_SG(ref TGarnetApi storageApi) { // Realized not-found without IO, and no earlier pending, so we can add directly to the output WriteNull(); - o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); } else { @@ -125,7 +125,7 @@ private bool NetworkMGET_SG(ref TGarnetApi storageApi) if (firstPending != -1) { // First complete all pending ops - storageApi.GET_CompletePending(outputArr, true); + _ = storageApi.GET_CompletePending(outputArr, true); // Write the outputs to network buffer for (var i = firstPending; i < parseState.Count; i++) @@ -161,9 +161,9 @@ private bool NetworkMSET(ref TGarnetApi storageApi) for (int c = 0; c < parseState.Count; c += 2) { - var key = parseState.GetArgSliceByRef(c).SpanByte; - var val = parseState.GetArgSliceByRef(c + 1).SpanByte; - _ = storageApi.SET(ref key, ref val); + var key = parseState.GetArgSliceByRef(c); + var val = parseState.GetArgSliceByRef(c + 1); + _ = storageApi.SET(key, val); } while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); @@ -196,8 +196,8 @@ private bool NetworkDEL(ref TGarnetApi storageApi) int keysDeleted = 0; for (int c = 0; c < parseState.Count; c++) { - var key = parseState.GetArgSliceByRef(c).SpanByte; - var status = storageApi.DELETE(ref key, StoreType.All); + var key = parseState.GetArgSliceByRef(c); + var status = storageApi.DELETE(key, StoreType.All); // 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) @@ -380,7 +380,7 @@ private bool NetworkSCAN(ref TGarnetApi storageApi) } var pattern = "*"u8; - var patternArgSlice = ArgSlice.FromPinnedSpan(pattern); + var patternArgSlice = PinnedSpanByte.FromPinnedSpan(pattern); var allKeys = true; long countValue = 10; ReadOnlySpan typeParameterValue = default; @@ -571,7 +571,7 @@ private bool NetworkLCS(ref TGarnetApi storageApi) return AbortWithErrorMessage(CmdStrings.RESP_ERR_LENGTH_AND_INDEXES); } - var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var output = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.LCS(key1, key2, ref output, lenOnly, withIndices, withMatchLen, minMatchLen); if (!output.IsSpanByte) diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs index 857b34e1979..e2aacbbbbb5 100644 --- a/libs/server/Resp/BasicCommands.cs +++ b/libs/server/Resp/BasicCommands.cs @@ -31,9 +31,9 @@ bool NetworkGET(ref TGarnetApi storageApi) RawStringInput input = default; - var key = parseState.GetArgSliceByRef(0).SpanByte; - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.GET(ref key, ref input, ref o); + var key = parseState.GetArgSliceByRef(0); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.GET(key, ref input, ref o); switch (status) { @@ -59,11 +59,9 @@ bool NetworkGETEX(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { if (parseState.Count < 1 || parseState.Count > 3) - { return AbortWithWrongNumberOfArguments(nameof(RespCommand.GETEX)); - } - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); TimeSpan? tsExpiry = null; if (parseState.Count > 1) @@ -111,8 +109,8 @@ bool NetworkGETEX(ref TGarnetApi storageApi) var expiry = (tsExpiry.HasValue && tsExpiry.Value.Ticks > 0) ? DateTimeOffset.UtcNow.Ticks + tsExpiry.Value.Ticks : 0; var input = new RawStringInput(RespCommand.GETEX, ref parseState, startIdx: 1, arg1: expiry); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.GETEX(ref key, ref input, ref o); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.GETEX(key, ref input, ref o); switch (status) { @@ -137,15 +135,15 @@ bool NetworkGETEX(ref TGarnetApi storageApi) bool NetworkGETAsync(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Optimistically ask storage to write output to network buffer - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); // Set up input to instruct storage to write output to IMemory rather than // network buffer, if the operation goes pending. var input = new RawStringInput(RespCommand.ASYNC); - var status = storageApi.GET_WithPending(ref key, ref input, ref o, asyncStarted, out var pending); + var status = storageApi.GET_WithPending(key, ref input, ref o, asyncStarted, out var pending); if (pending) { @@ -176,11 +174,11 @@ bool NetworkGETAsync(ref TGarnetApi storageApi) bool NetworkGET_SG(ref TGarnetApi storageApi) where TGarnetApi : IGarnetAdvancedApi { - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); RawStringInput input = default; var firstPending = -1; (GarnetStatus, SpanByteAndMemory)[] outputArr = null; - SpanByteAndMemory o = new(dcurr, (int)(dend - dcurr)); + SpanByteAndMemory o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var c = 0; for (; ; c++) @@ -191,7 +189,7 @@ bool NetworkGET_SG(ref TGarnetApi storageApi) // Store index in context, since completions are not in order long ctx = firstPending == -1 ? 0 : c - firstPending; - var status = storageApi.GET_WithPending(ref key, ref input, ref o, ctx, + var status = storageApi.GET_WithPending(key, ref input, ref o, ctx, out var isPending); if (isPending) @@ -210,7 +208,7 @@ bool NetworkGET_SG(ref TGarnetApi storageApi) SendAndReset(o.Memory, o.Length); else dcurr += o.Length; - o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); } else { @@ -224,7 +222,7 @@ bool NetworkGET_SG(ref TGarnetApi storageApi) { // Realized not-found without IO, and no earlier pending, so we can add directly to the output WriteNull(); - o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); } else { @@ -238,7 +236,7 @@ bool NetworkGET_SG(ref TGarnetApi storageApi) if (firstPending != -1) { // First complete all pending ops - storageApi.GET_CompletePending(outputArr, true); + _ = storageApi.GET_CompletePending(outputArr, true); // Write the outputs to network buffer for (var i = 0; i < c - firstPending; i++) @@ -280,10 +278,10 @@ private bool NetworkSET(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { Debug.Assert(parseState.Count == 2); - var key = parseState.GetArgSliceByRef(0).SpanByte; - var value = parseState.GetArgSliceByRef(1).SpanByte; + var key = parseState.GetArgSliceByRef(0); + var value = parseState.GetArgSliceByRef(1); - storageApi.SET(ref key, ref value); + _ = storageApi.SET(key, value); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); @@ -298,10 +296,9 @@ private bool NetworkGETSET(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { Debug.Assert(parseState.Count == 2); - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); - return NetworkSET_Conditional(RespCommand.SET, 0, ref key, true, - false, false, ref storageApi); + return NetworkSET_Conditional(RespCommand.SET, 0, key, true, false, false, ref storageApi); } /// @@ -330,9 +327,9 @@ private bool NetworkSetRange(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.SETRANGE, ref parseState, startIdx: 1); Span outputBuffer = stackalloc byte[NumUtils.MaximumFormatInt64Length]; - var output = ArgSlice.FromPinnedSpan(outputBuffer); + var output = PinnedSpanByte.FromPinnedSpan(outputBuffer); - storageApi.SETRANGE(key, ref input, ref output); + _ = storageApi.SETRANGE(key, ref input, ref output); while (!RespWriteUtils.TryWriteIntegerFromBytes(outputBuffer.Slice(0, output.Length), ref dcurr, dend)) SendAndReset(); @@ -344,7 +341,6 @@ private bool NetworkGetRange(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { var key = parseState.GetArgSliceByRef(0); - var sbKey = key.SpanByte; // Validate range if (!parseState.TryGetInt(1, out _) || !parseState.TryGetInt(2, out _)) @@ -356,9 +352,9 @@ private bool NetworkGetRange(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.GETRANGE, ref parseState, startIdx: 1); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.GETRANGE(ref sbKey, ref input, ref o); + var status = storageApi.GETRANGE(key, ref input, ref o); if (status == GarnetStatus.OK) { @@ -385,7 +381,7 @@ private bool NetworkGetRange(ref TGarnetApi storageApi) private bool NetworkSETEX(bool highPrecision, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Validate expiry if (!parseState.TryGetInt(1, out var expiry)) @@ -407,10 +403,10 @@ private bool NetworkSETEX(bool highPrecision, ref TGarnetApi storage ? TimeSpan.FromMilliseconds(expiry).Ticks : TimeSpan.FromSeconds(expiry).Ticks); - var sbVal = parseState.GetArgSliceByRef(2).SpanByte; + var value = parseState.GetArgSliceByRef(2); var input = new RawStringInput(RespCommand.SETEX, 0, valMetadata); - _ = storageApi.SET(ref key, ref input, ref sbVal); + _ = storageApi.SET(key, ref input, value); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); @@ -430,10 +426,9 @@ private bool NetworkSETNX(bool highPrecision, ref TGarnetApi storage } var key = parseState.GetArgSliceByRef(0); - var sbKey = key.SpanByte; var input = new RawStringInput(RespCommand.SETEXNX, ref parseState, startIdx: 1); - var status = storageApi.SET_Conditional(ref sbKey, ref input); + var status = storageApi.SET_Conditional(key, ref input); // The status returned for SETNX as NOTFOUND is the expected status in the happy path var retVal = status == GarnetStatus.NOTFOUND ? 1 : 0; @@ -450,10 +445,7 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { var key = parseState.GetArgSliceByRef(0); - var sbKey = key.SpanByte; - var val = parseState.GetArgSliceByRef(1); - var sbVal = val.SpanByte; var expiry = 0; ReadOnlySpan errorMessage = default; @@ -585,15 +577,12 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi) { case ExistOptions.None: return getValue || withEtag - ? NetworkSET_Conditional(RespCommand.SET, expiry, ref sbKey, getValue, - isHighPrecision, withEtag, ref storageApi) - : NetworkSET_EX(RespCommand.SET, expOption, expiry, ref sbKey, ref sbVal, ref storageApi); // Can perform a blind update + ? NetworkSET_Conditional(RespCommand.SET, expiry, key, getValue, isHighPrecision, withEtag, ref storageApi) + : NetworkSET_EX(RespCommand.SET, expOption, expiry, key, val, ref storageApi); // Can perform a blind update case ExistOptions.XX: - return NetworkSET_Conditional(RespCommand.SETEXXX, expiry, ref sbKey, - getValue, isHighPrecision, withEtag, ref storageApi); + return NetworkSET_Conditional(RespCommand.SETEXXX, expiry, key, getValue, isHighPrecision, withEtag, ref storageApi); case ExistOptions.NX: - return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, ref sbKey, - getValue, isHighPrecision, withEtag, ref storageApi); + return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, key, getValue, isHighPrecision, withEtag, ref storageApi); } break; case ExpirationOption.KEEPTTL: @@ -602,14 +591,11 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi) { case ExistOptions.None: // We can never perform a blind update due to KEEPTTL - return NetworkSET_Conditional(RespCommand.SETKEEPTTL, expiry, ref sbKey - , getValue, highPrecision: false, withEtag, ref storageApi); + return NetworkSET_Conditional(RespCommand.SETKEEPTTL, expiry, key, getValue, highPrecision: false, withEtag, ref storageApi); case ExistOptions.XX: - return NetworkSET_Conditional(RespCommand.SETKEEPTTLXX, expiry, ref sbKey, - getValue, highPrecision: false, withEtag, ref storageApi); + return NetworkSET_Conditional(RespCommand.SETKEEPTTLXX, expiry, key, getValue, highPrecision: false, withEtag, ref storageApi); case ExistOptions.NX: - return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, ref sbKey, - getValue, highPrecision: false, withEtag, ref storageApi); + return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, key, getValue, highPrecision: false, withEtag, ref storageApi); } break; } @@ -619,8 +605,7 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi) return true; } - private unsafe bool NetworkSET_EX(RespCommand cmd, ExpirationOption expOption, int expiry, - ref SpanByte key, ref SpanByte val, ref TGarnetApi storageApi) + private unsafe bool NetworkSET_EX(RespCommand cmd, ExpirationOption expOption, int expiry, PinnedSpanByte key, PinnedSpanByte val, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { Debug.Assert(cmd == RespCommand.SET); @@ -633,14 +618,14 @@ private unsafe bool NetworkSET_EX(RespCommand cmd, ExpirationOption var input = new RawStringInput(cmd, 0, valMetadata); - storageApi.SET(ref key, ref input, ref val); + storageApi.SET(key, ref input, val); while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); return true; } - private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref SpanByte key, bool getValue, bool highPrecision, bool withEtag, ref TGarnetApi storageApi) + private bool NetworkSET_Conditional(RespCommand cmd, int expiry, PinnedSpanByte key, bool getValue, bool highPrecision, bool withEtag, ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { var inputArg = expiry == 0 @@ -654,10 +639,10 @@ private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref if (!getValue && !withEtag) { - // the following debug assertion is the catch any edge case leading to SETIFMATCH, or SETIFGREATER skipping the above block + // the following debug assertion is to catch any edge case leading to SETIFMATCH, or SETIFGREATER skipping the above block Debug.Assert(cmd is not (RespCommand.SETIFMATCH or RespCommand.SETIFGREATER), "SETIFMATCH should have gone though pointing to right output variable"); - var status = storageApi.SET_Conditional(ref key, ref input); + var status = storageApi.SET_Conditional(key, ref input); // KEEPTTL without flags doesn't care whether it was found or not. if (cmd == RespCommand.SETKEEPTTL) @@ -689,14 +674,14 @@ private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref else { if (withEtag) - input.header.SetWithEtagFlag(); + input.header.SetWithETagFlag(); if (getValue) input.header.SetSetGetFlag(); // anything with getValue or withEtag always writes to the buffer in the happy path - SpanByteAndMemory outputBuffer = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - GarnetStatus status = storageApi.SET_Conditional(ref key, ref input, ref outputBuffer); + SpanByteAndMemory outputBuffer = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + GarnetStatus status = storageApi.SET_Conditional(key, ref input, ref outputBuffer); // The data will be on the buffer either when we know the response is ok or when the withEtag flag is set. bool ok = status != GarnetStatus.NOTFOUND || withEtag; @@ -741,10 +726,10 @@ private bool NetworkIncrement(RespCommand cmd, ref TGarnetApi storag } Span outputBuffer = stackalloc byte[NumUtils.MaximumFormatInt64Length + 1]; - var output = ArgSlice.FromPinnedSpan(outputBuffer); + var output = PinnedSpanByte.FromPinnedSpan(outputBuffer); var input = new RawStringInput(cmd, 0, incrByValue); - storageApi.Increment(key, ref input, ref output); + _ = storageApi.Increment(key, ref input, ref output); var errorFlag = output.Length == NumUtils.MaximumFormatInt64Length + 1 ? (OperationError)output.Span[0] @@ -784,10 +769,10 @@ private bool NetworkIncrementByFloat(ref TGarnetApi storageApi) } Span outputBuffer = stackalloc byte[NumUtils.MaximumFormatDoubleLength + 1]; - var output = ArgSlice.FromPinnedSpan(outputBuffer); + var output = PinnedSpanByte.FromPinnedSpan(outputBuffer); var input = new RawStringInput(RespCommand.INCRBYFLOAT, ref parseState, startIdx: 1); - storageApi.Increment(key, ref input, ref output); + _ = storageApi.Increment(key, ref input, ref output); var errorFlag = output.Length == NumUtils.MaximumFormatDoubleLength + 1 ? (OperationError)output.Span[0] @@ -817,14 +802,14 @@ private bool NetworkIncrementByFloat(ref TGarnetApi storageApi) private bool NetworkAppend(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var sbKey = parseState.GetArgSliceByRef(0); var input = new RawStringInput(RespCommand.APPEND, ref parseState, startIdx: 1); Span outputBuffer = stackalloc byte[NumUtils.MaximumFormatInt64Length]; var output = SpanByteAndMemory.FromPinnedSpan(outputBuffer); - storageApi.APPEND(ref sbKey, ref input, ref output); + storageApi.APPEND(sbKey, ref input, ref output); while (!RespWriteUtils.TryWriteIntegerFromBytes(outputBuffer.Slice(0, output.Length), ref dcurr, dend)) SendAndReset(); @@ -962,7 +947,7 @@ private bool NetworkSTRLEN(ref TGarnetApi storageApi) //STRLEN key var key = parseState.GetArgSliceByRef(0); - var status = storageApi.GET(key, out var value); + var status = storageApi.GET(key, out PinnedSpanByte value); switch (status) { @@ -984,7 +969,7 @@ private bool NetworkSTRLEN(ref TGarnetApi storageApi) /// private void WriteCOMMANDResponse() { - var spam = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var spam = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var writer = new RespMemoryWriter(respProtocolVersion, ref spam); try @@ -1081,7 +1066,7 @@ private bool NetworkCOMMAND_DOCS() { var count = parseState.Count; - var spam = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var spam = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var writer = new RespMemoryWriter(respProtocolVersion, ref spam); try @@ -1155,7 +1140,7 @@ private bool NetworkCOMMAND_INFO() } else { - var spam = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var spam = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var writer = new RespMemoryWriter(respProtocolVersion, ref spam); try @@ -1778,7 +1763,7 @@ private static void WriteClientInfo(IClusterProvider provider, StringBuilder int into.Append($" lib-ver={targetSession.clientLibVersion}"); } - bool ParseGETAndKey(ref SpanByte key) + bool ParseGETAndKey(ref PinnedSpanByte key) { var oldEndReadHead = readHead = endReadHead; var cmd = ParseCommand(writeErrorOnFailure: true, out var success); @@ -1788,7 +1773,7 @@ bool ParseGETAndKey(ref SpanByte key) endReadHead = readHead = oldEndReadHead; return false; } - key = parseState.GetArgSliceByRef(0).SpanByte; + key = parseState.GetArgSliceByRef(0); return true; } @@ -1817,7 +1802,7 @@ static void SetResult(int c, ref int firstPending, ref (GarnetStatus, SpanByteAn outputArr[c - firstPending] = (status, output); } - static long NextPowerOf2(long v) + static long NextPowerOf2(long v) // TODO: consolidate this and other Tsavorite.core.Utility method clones { v--; v |= v >> 1; diff --git a/libs/server/Resp/BasicEtagCommands.cs b/libs/server/Resp/BasicEtagCommands.cs index 6e9c35fc22b..ec5d8b25afb 100644 --- a/libs/server/Resp/BasicEtagCommands.cs +++ b/libs/server/Resp/BasicEtagCommands.cs @@ -22,10 +22,10 @@ private bool NetworkGETWITHETAG(ref TGarnetApi storageApi) { Debug.Assert(parseState.Count == 1); - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); var input = new RawStringInput(RespCommand.GETWITHETAG); - var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.GET(ref key, ref input, ref output); + var output = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.GET(key, ref input, ref output); switch (status) { @@ -53,10 +53,10 @@ private bool NetworkGETIFNOTMATCH(ref TGarnetApi storageApi) { Debug.Assert(parseState.Count == 2); - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); var input = new RawStringInput(RespCommand.GETIFNOTMATCH, ref parseState, startIdx: 1); - var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.GET(ref key, ref input, ref output); + var output = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.GET(key, ref input, ref output); switch (status) { @@ -87,7 +87,7 @@ private bool NetworkDELIFGREATER(ref TGarnetApi storageApi) if (parseState.Count != 2) return AbortWithWrongNumberOfArguments(nameof(RespCommand.DELIFGREATER)); - SpanByte key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); if (!parseState.TryGetLong(1, out long givenEtag) || givenEtag < 0) { while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_INVALID_ETAG, ref dcurr, dend)) @@ -99,9 +99,9 @@ private bool NetworkDELIFGREATER(ref TGarnetApi storageApi) // To achieve this, we use a conditional DEL command to gain RMW (Read-Modify-Write) access, enabling deletion based on conditions. RawStringInput input = new RawStringInput(RespCommand.DELIFGREATER, ref parseState, startIdx: 1); - input.header.SetWithEtagFlag(); + input.header.SetWithETagFlag(); - GarnetStatus status = storageApi.DEL_Conditional(ref key, ref input); + GarnetStatus status = storageApi.DEL_Conditional(key, ref input); int keysDeleted = status == GarnetStatus.OK ? 1 : 0; @@ -215,10 +215,8 @@ private bool NetworkSetETagConditional(RespCommand cmd, ref TGarnetA return true; } - SpanByte key = parseState.GetArgSliceByRef(0).SpanByte; - - NetworkSET_Conditional(cmd, expiry, ref key, getValue: !noGet, highPrecision: expOption == ExpirationOption.PX, withEtag: true, ref storageApi); - + var key = parseState.GetArgSliceByRef(0); + NetworkSET_Conditional(cmd, expiry, key, getValue: !noGet, highPrecision: expOption == ExpirationOption.PX, withEtag: true, ref storageApi); return true; } } diff --git a/libs/server/Resp/Bitmap/BitmapCommands.cs b/libs/server/Resp/Bitmap/BitmapCommands.cs index e138cffcd40..78d2b58d124 100644 --- a/libs/server/Resp/Bitmap/BitmapCommands.cs +++ b/libs/server/Resp/Bitmap/BitmapCommands.cs @@ -9,7 +9,7 @@ namespace Garnet.server { - using SecondaryCommandList = List<(RespCommand, ArgSlice[])>; + using SecondaryCommandList = List<(RespCommand, PinnedSpanByte[])>; /// (1) , (2) , (3) /// overflow check, ptr protection, and status not found implemented for below @@ -130,7 +130,7 @@ private bool NetworkStringSetBit(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments(nameof(RespCommand.SETBIT)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Validate offset if (!parseState.TryGetLong(1, out var offset) || (offset < 0)) @@ -153,11 +153,8 @@ private bool NetworkStringSetBit(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.SETBIT, ref parseState, startIdx: 1, arg1: offset); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.StringSetBit( - ref sbKey, - ref input, - ref o); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.StringSetBit(key, ref input, ref o); if (status == GarnetStatus.OK) dcurr += o.Length; @@ -176,7 +173,7 @@ private bool NetworkStringGetBit(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments(nameof(RespCommand.GETBIT)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Validate offset if (!parseState.TryGetLong(1, out var offset) || (offset < 0)) @@ -189,8 +186,8 @@ private bool NetworkStringGetBit(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.GETBIT, ref parseState, startIdx: 1, arg1: offset); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.StringGetBit(ref sbKey, ref input, ref o); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.StringGetBit(key, ref input, ref o); if (status == GarnetStatus.NOTFOUND) while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_RETURN_VAL_0, ref dcurr, dend)) @@ -210,12 +207,10 @@ private bool NetworkStringBitCount(ref TGarnetApi storageApi) { var count = parseState.Count; if (count < 1 || count > 4) - { return AbortWithWrongNumberOfArguments(nameof(RespCommand.BITCOUNT)); - } // <[Get Key]> - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Validate start & end offsets, if exist if (parseState.Count > 1) @@ -231,9 +226,9 @@ private bool NetworkStringBitCount(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.BITCOUNT, ref parseState, startIdx: 1); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.StringBitCount(ref sbKey, ref input, ref o); + var status = storageApi.StringBitCount(key, ref input, ref o); if (status == GarnetStatus.OK) { @@ -259,12 +254,10 @@ private bool NetworkStringBitPosition(ref TGarnetApi storageApi) { var count = parseState.Count; if (count < 2 || count > 5) - { return AbortWithWrongNumberOfArguments(nameof(RespCommand.BITPOS)); - } // <[Get Key]> - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); // Validate value var bSetValSlice = parseState.GetArgSliceByRef(1).ReadOnlySpan; @@ -305,9 +298,9 @@ private bool NetworkStringBitPosition(ref TGarnetApi storageApi) var input = new RawStringInput(RespCommand.BITPOS, ref parseState, startIdx: 1); - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.StringBitPosition(ref sbKey, ref input, ref o); + var status = storageApi.StringBitPosition(key, ref input, ref o); if (status == GarnetStatus.OK) { @@ -370,10 +363,10 @@ private bool StringBitField(ref TGarnetApi storageApi) // BITFIELD key [GET encoding offset] [SET encoding offset value] [INCRBY encoding offset increment] [OVERFLOW WRAP| SAT | FAIL] // Extract Key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); var isOverflowTypeSet = false; - ArgSlice overflowTypeSlice = default; + PinnedSpanByte overflowTypeSlice = default; var secondaryCommandArgs = new SecondaryCommandList(); var currTokenIdx = 1; @@ -459,7 +452,7 @@ private bool StringBitField(ref TGarnetApi storageApi) } } - return StringBitFieldAction(ref storageApi, ref sbKey, RespCommand.BITFIELD, + return StringBitFieldAction(ref storageApi, key, RespCommand.BITFIELD, secondaryCommandArgs, isOverflowTypeSet, overflowTypeSlice); } @@ -476,7 +469,7 @@ private bool StringBitFieldReadOnly(ref TGarnetApi storageApi) // BITFIELD_RO key [GET encoding offset [GET encoding offset] ... ] // Extract Key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); var secondaryCommandArgs = new SecondaryCommandList(); @@ -520,15 +513,15 @@ private bool StringBitFieldReadOnly(ref TGarnetApi storageApi) secondaryCommandArgs.Add((RespCommand.GET, [commandSlice, encodingSlice, offsetSlice])); } - return StringBitFieldAction(ref storageApi, ref sbKey, RespCommand.BITFIELD_RO, secondaryCommandArgs); + return StringBitFieldAction(ref storageApi, key, RespCommand.BITFIELD_RO, secondaryCommandArgs); } private bool StringBitFieldAction(ref TGarnetApi storageApi, - ref SpanByte sbKey, + PinnedSpanByte sbKey, RespCommand cmd, SecondaryCommandList secondaryCommandArgs, bool isOverflowTypeSet = false, - ArgSlice overflowTypeSlice = default) + PinnedSpanByte overflowTypeSlice = default) where TGarnetApi : IGarnetApi { while (!RespWriteUtils.TryWriteArrayLength(secondaryCommandArgs.Count, ref dcurr, dend)) @@ -554,8 +547,8 @@ private bool StringBitFieldAction(ref TGarnetApi storageApi, input.parseState = parseState; - var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = storageApi.StringBitField(ref sbKey, ref input, opCode, + var output = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.StringBitField(sbKey, ref input, opCode, ref output); if (status == GarnetStatus.NOTFOUND && opCode == RespCommand.GET) diff --git a/libs/server/Resp/ClientCommands.cs b/libs/server/Resp/ClientCommands.cs index 5f4bc2a9b27..792d4096aea 100644 --- a/libs/server/Resp/ClientCommands.cs +++ b/libs/server/Resp/ClientCommands.cs @@ -264,13 +264,13 @@ private bool NetworkCLIENTKILL() var filterSpan = filter.Span; var valueIx = argIx + 1; - ref var value = ref parseState.GetArgSliceByRef(valueIx); + var value = parseState.GetArgSliceByRef(valueIx); AsciiUtils.ToUpperInPlace(filterSpan); if (filterSpan.SequenceEqual(CmdStrings.ID)) { - if (!ParseUtils.TryReadLong(ref value, out var idParsed)) + if (!ParseUtils.TryReadLong(value, out var idParsed)) { return AbortWithErrorMessage(Encoding.ASCII.GetBytes(string.Format(CmdStrings.GenericErrShouldBeGreaterThanZero, "client-id"))); } @@ -291,7 +291,7 @@ private bool NetworkCLIENTKILL() if (!parseState.TryGetClientType(valueIx, out var typeParsed)) { - var typeStr = ParseUtils.ReadString(ref value); + var typeStr = ParseUtils.ReadString(value); return AbortWithErrorMessage(Encoding.UTF8.GetBytes(string.Format(CmdStrings.GenericUnknownClientType, typeStr))); } @@ -307,7 +307,7 @@ private bool NetworkCLIENTKILL() return AbortWithErrorMessage(Encoding.ASCII.GetBytes(string.Format(CmdStrings.GenericErrDuplicateFilter, "USER"))); } - user = ParseUtils.ReadString(ref value); + user = ParseUtils.ReadString(value); } else if (filterSpan.SequenceEqual(CmdStrings.ADDR)) { @@ -316,7 +316,7 @@ private bool NetworkCLIENTKILL() return AbortWithErrorMessage(Encoding.ASCII.GetBytes(string.Format(CmdStrings.GenericErrDuplicateFilter, "ADDR"))); } - addr = ParseUtils.ReadString(ref value); + addr = ParseUtils.ReadString(value); } else if (filterSpan.SequenceEqual(CmdStrings.LADDR)) { @@ -325,7 +325,7 @@ private bool NetworkCLIENTKILL() return AbortWithErrorMessage(Encoding.ASCII.GetBytes(string.Format(CmdStrings.GenericErrDuplicateFilter, "LADDR"))); } - lAddr = ParseUtils.ReadString(ref value); + lAddr = ParseUtils.ReadString(value); } else if (filterSpan.SequenceEqual(CmdStrings.SKIPME)) { @@ -351,7 +351,7 @@ private bool NetworkCLIENTKILL() } else if (filterSpan.SequenceEqual(CmdStrings.MAXAGE)) { - if (!ParseUtils.TryReadLong(ref value, out var maxAgeParsed)) + if (!ParseUtils.TryReadLong(value, out var maxAgeParsed)) { return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); } diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs index 0e52d40d9c1..eebe240656b 100644 --- a/libs/server/Resp/GarnetDatabaseSession.cs +++ b/libs/server/Resp/GarnetDatabaseSession.cs @@ -3,18 +3,18 @@ namespace Garnet.server { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; - using LockableGarnetApi = GarnetApi, - SpanByteAllocator>>, - LockableContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; + using TransactionalGarnetApi = GarnetApi, + SpanByteAllocator>>, + TransactionalContext, + ObjectAllocator>>>; /// /// Represents a logical database session in Garnet @@ -39,7 +39,7 @@ internal class GarnetDatabaseSession : IDisposable /// /// Lockable Garnet API /// - public LockableGarnetApi LockableGarnetApi { get; } + public TransactionalGarnetApi TransactionalGarnetApi { get; } /// /// Transaction manager @@ -48,12 +48,12 @@ internal class GarnetDatabaseSession : IDisposable bool disposed = false; - public GarnetDatabaseSession(int id, StorageSession storageSession, BasicGarnetApi garnetApi, LockableGarnetApi lockableGarnetApi, TransactionManager txnManager) + public GarnetDatabaseSession(int id, StorageSession storageSession, BasicGarnetApi garnetApi, TransactionalGarnetApi lockableGarnetApi, TransactionManager txnManager) { this.Id = id; this.StorageSession = storageSession; this.GarnetApi = garnetApi; - this.LockableGarnetApi = lockableGarnetApi; + this.TransactionalGarnetApi = lockableGarnetApi; this.TransactionManager = txnManager; } @@ -62,7 +62,7 @@ public GarnetDatabaseSession(int id, GarnetDatabaseSession srcSession) this.Id = id; this.StorageSession = srcSession.StorageSession; this.GarnetApi = srcSession.GarnetApi; - this.LockableGarnetApi = srcSession.LockableGarnetApi; + this.TransactionalGarnetApi = srcSession.TransactionalGarnetApi; this.TransactionManager = srcSession.TransactionManager; } diff --git a/libs/server/Resp/HyperLogLog/HyperLogLog.cs b/libs/server/Resp/HyperLogLog/HyperLogLog.cs index 3ee7ca037a3..ba944f6346e 100644 --- a/libs/server/Resp/HyperLogLog/HyperLogLog.cs +++ b/libs/server/Resp/HyperLogLog/HyperLogLog.cs @@ -605,7 +605,7 @@ private bool IterateUpdate(ref RawStringInput input, byte* value, bool dense) for (var i = 0; i < input.parseState.Count; i++) { var currElement = input.parseState.GetArgSliceByRef(i); - var hashValue = (long)HashUtils.MurmurHash2x64A(currElement.ptr, currElement.Length); + var hashValue = (long)HashUtils.MurmurHash2x64A(currElement.ToPointer(), currElement.Length); updated |= (dense ? UpdateDense(value, hashValue) : UpdateSparse(value, hashValue)); } return updated; diff --git a/libs/server/Resp/HyperLogLog/HyperLogLogCommands.cs b/libs/server/Resp/HyperLogLog/HyperLogLogCommands.cs index 3ccff292178..4fe90c1343d 100644 --- a/libs/server/Resp/HyperLogLog/HyperLogLogCommands.cs +++ b/libs/server/Resp/HyperLogLog/HyperLogLogCommands.cs @@ -28,13 +28,13 @@ private bool HyperLogLogAdd(ref TGarnetApi storageApi) var output = stackalloc byte[1]; byte pfaddUpdated = 0; - var key = parseState.GetArgSliceByRef(0).SpanByte; + var key = parseState.GetArgSliceByRef(0); for (var i = 1; i < parseState.Count; i++) { input.parseState = parseState.Slice(i, 1); - var o = new SpanByteAndMemory(output, 1); - storageApi.HyperLogLogAdd(ref key, ref input, ref o); + var o = SpanByteAndMemory.FromPinnedPointer(output, 1); + storageApi.HyperLogLogAdd(key, ref input, ref o); // Invalid HLL Type if (*output == 0xFF) diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs index 1f28d1f272d..7c2abf5cd32 100644 --- a/libs/server/Resp/KeyAdminCommands.cs +++ b/libs/server/Resp/KeyAdminCommands.cs @@ -98,11 +98,8 @@ bool NetworkRESTORE(ref TGarnetApi storageApi) // Start from payload start and skip the value type byte var val = value.ReadOnlySpan.Slice(payloadStart + 1, length); - var valArgSlice = scratchBufferManager.CreateArgSlice(val); - var sbKey = key.SpanByte; - parseState.InitializeWithArgument(valArgSlice); RawStringInput input; @@ -116,7 +113,7 @@ bool NetworkRESTORE(ref TGarnetApi storageApi) input = new RawStringInput(RespCommand.SETEXNX, ref parseState); } - var status = storageApi.SET_Conditional(ref sbKey, ref input); + var status = storageApi.SET_Conditional(key, ref input); if (status is GarnetStatus.NOTFOUND) { @@ -144,7 +141,7 @@ bool NetworkDUMP(ref TGarnetApi storageApi) var key = parseState.GetArgSliceByRef(0); - var status = storageApi.GET(key, out var value); + var status = storageApi.GET(key, out PinnedSpanByte value); if (status is GarnetStatus.NOTFOUND) { @@ -193,7 +190,9 @@ bool NetworkDUMP(ref TGarnetApi storageApi) offset += encodedLength.Length; // copy value to buffer - value.ReadOnlySpan.CopyTo(buffer[offset..]); + value. + // copy value to buffer + ReadOnlySpan.CopyTo(buffer[offset..]); offset += value.ReadOnlySpan.Length; // Write RDB version @@ -329,9 +328,9 @@ private bool NetworkGETDEL(ref TGarnetApi garnetApi) return AbortWithWrongNumberOfArguments(nameof(RespCommand.GETDEL)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); - var status = garnetApi.GETDEL(ref sbKey, ref o); + var sbKey = parseState.GetArgSliceByRef(0); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = garnetApi.GETDEL(sbKey, ref o); if (status == GarnetStatus.OK) { @@ -597,11 +596,11 @@ private bool NetworkTTL(RespCommand command, ref TGarnetApi storageA return AbortWithWrongNumberOfArguments(nameof(RespCommand.PERSIST)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var key = parseState.GetArgSliceByRef(0); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = command == RespCommand.TTL ? - storageApi.TTL(ref sbKey, StoreType.All, ref o) : - storageApi.PTTL(ref sbKey, StoreType.All, ref o); + storageApi.TTL(key, StoreType.All, ref o) : + storageApi.PTTL(key, StoreType.All, ref o); if (status == GarnetStatus.OK) { @@ -633,11 +632,11 @@ private bool NetworkEXPIRETIME(RespCommand command, ref TGarnetApi s return AbortWithWrongNumberOfArguments(nameof(RespCommand.EXPIRETIME)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var key = parseState.GetArgSliceByRef(0); + var o = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = command == RespCommand.EXPIRETIME ? - storageApi.EXPIRETIME(ref sbKey, StoreType.All, ref o) : - storageApi.PEXPIRETIME(ref sbKey, StoreType.All, ref o); + storageApi.EXPIRETIME(key, StoreType.All, ref o) : + storageApi.PEXPIRETIME(key, StoreType.All, ref o); if (status == GarnetStatus.OK) { diff --git a/libs/server/Resp/LocalServerSession.cs b/libs/server/Resp/LocalServerSession.cs index 9c466af8235..4f716e1d14c 100644 --- a/libs/server/Resp/LocalServerSession.cs +++ b/libs/server/Resp/LocalServerSession.cs @@ -7,12 +7,12 @@ namespace Garnet.server { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; /// /// Local server session diff --git a/libs/server/Resp/Objects/HashCommands.cs b/libs/server/Resp/Objects/HashCommands.cs index f1fc77b8109..f553c4631f8 100644 --- a/libs/server/Resp/Objects/HashCommands.cs +++ b/libs/server/Resp/Objects/HashCommands.cs @@ -31,8 +31,7 @@ private unsafe bool HashSet(RespCommand command, ref TGarnetApi stor return AbortWithWrongNumberOfArguments(command.ToString()); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var hop = command switch @@ -48,7 +47,7 @@ private unsafe bool HashSet(RespCommand command, ref TGarnetApi stor var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.HashSet(keyBytes, ref input, out var output); + var status = storageApi.HashSet(key, ref input, out var output); switch (status) { @@ -86,17 +85,16 @@ private bool HashGet(RespCommand command, ref TGarnetApi storageApi) if (parseState.Count != 2) return AbortWithWrongNumberOfArguments(command.ToString()); - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HGET }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.HashGet(keyBytes, ref input, ref output); + var status = storageApi.HashGet(key, ref input, ref output); switch (status) { @@ -129,17 +127,16 @@ private bool HashGetAll(RespCommand command, ref TGarnetApi storageA return AbortWithWrongNumberOfArguments(command.ToString()); // Get the hash key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HGETALL }; var input = new ObjectInput(header, respProtocolVersion); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.HashGetAll(keyBytes, ref input, ref output); + var status = storageApi.HashGetAll(key, ref input, ref output); switch (status) { @@ -172,8 +169,7 @@ private bool HashGetMultiple(RespCommand command, ref TGarnetApi sto if (parseState.Count < 2) return AbortWithWrongNumberOfArguments(command.ToString()); - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HMGET }; @@ -181,9 +177,9 @@ private bool HashGetMultiple(RespCommand command, ref TGarnetApi sto var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.HashGetMultiple(keyBytes, ref input, ref output); + var status = storageApi.HashGetMultiple(key, ref input, ref output); switch (status) { @@ -220,8 +216,7 @@ private bool HashRandomField(RespCommand command, ref TGarnetApi sto if (parseState.Count < 1 || parseState.Count > 3) return AbortWithWrongNumberOfArguments(command.ToString()); - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var paramCount = 1; var withValues = false; @@ -264,7 +259,7 @@ private bool HashRandomField(RespCommand command, ref TGarnetApi sto var input = new ObjectInput(header, countWithMetadata, seed); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = GarnetStatus.NOTFOUND; @@ -272,8 +267,8 @@ private bool HashRandomField(RespCommand command, ref TGarnetApi sto if (paramCount != 0) { // Prepare GarnetObjectStore output - output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); - status = storageApi.HashRandomField(keyBytes, ref input, ref output); + output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + status = storageApi.HashRandomField(key, ref input, ref output); } switch (status) @@ -315,15 +310,14 @@ private unsafe bool HashLength(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("HLEN"); } - // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + // Get the key + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HLEN }; var input = new ObjectInput(header); - var status = storageApi.HashLength(keyBytes, ref input, out var output); + var status = storageApi.HashLength(key, ref input, out var output); switch (status) { @@ -359,14 +353,13 @@ private unsafe bool HashStrLength(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("HSTRLEN"); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HSTRLEN }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.HashStrLength(keyBytes, ref input, out var output); + var status = storageApi.HashStrLength(key, ref input, out var output); switch (status) { @@ -402,14 +395,13 @@ private unsafe bool HashDelete(ref TGarnetApi storageApi) } // Get the key for Hash - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HDEL }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.HashDelete(keyBytes, ref input, out var output); + var status = storageApi.HashDelete(key, ref input, out var output); switch (status) { @@ -443,14 +435,13 @@ private unsafe bool HashExists(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("HEXISTS"); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HEXISTS }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.HashExists(keyBytes, ref input, out var output); + var status = storageApi.HashExists(key, ref input, out var output); switch (status) { @@ -487,8 +478,7 @@ private unsafe bool HashKeys(RespCommand command, ref TGarnetApi sto } // Get the key for Hash - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var op = command switch @@ -503,11 +493,11 @@ private unsafe bool HashKeys(RespCommand command, ref TGarnetApi sto var input = new ObjectInput(header); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = command == RespCommand.HKEYS - ? storageApi.HashKeys(keyBytes, ref input, ref output) - : storageApi.HashVals(keyBytes, ref input, ref output); + ? storageApi.HashKeys(key, ref input, ref output) + : storageApi.HashVals(key, ref input, ref output); switch (status) { @@ -545,8 +535,7 @@ private unsafe bool HashIncrement(RespCommand command, ref TGarnetAp } // Get the key for Hash - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var op = command switch @@ -561,9 +550,9 @@ private unsafe bool HashIncrement(RespCommand command, ref TGarnetAp var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.HashIncrement(keyBytes, ref input, ref output); + var status = storageApi.HashIncrement(key, ref input, ref output); switch (status) { @@ -655,7 +644,7 @@ private unsafe bool HashExpire(RespCommand command, ref TGarnetApi s var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HEXPIRE }; var input = new ObjectInput(header, ref fieldsParseState); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.HashExpire(key, expireAt, isMilliseconds, expireOption, ref input, ref output); @@ -745,7 +734,7 @@ private unsafe bool HashTimeToLive(RespCommand command, ref TGarnetA var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HTTL }; var input = new ObjectInput(header, ref fieldsParseState); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.HashTimeToLive(key, isMilliseconds, isTimestamp, ref input, ref output); @@ -807,7 +796,7 @@ private unsafe bool HashPersist(ref TGarnetApi storageApi) var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HPERSIST }; var input = new ObjectInput(header, ref fieldsParseState); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.HashPersist(key, ref input, ref output); diff --git a/libs/server/Resp/Objects/ListCommands.cs b/libs/server/Resp/Objects/ListCommands.cs index 44cd74d7a8b..de4af40a2ce 100644 --- a/libs/server/Resp/Objects/ListCommands.cs +++ b/libs/server/Resp/Objects/ListCommands.cs @@ -4,6 +4,7 @@ using System; using System.Text; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -25,8 +26,7 @@ private unsafe bool ListPush(RespCommand command, ref TGarnetApi sto return AbortWithWrongNumberOfArguments(command.ToString()); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var lop = command switch @@ -43,8 +43,8 @@ private unsafe bool ListPush(RespCommand command, ref TGarnetApi sto var input = new ObjectInput(header, ref parseState, startIdx: 1); var status = command == RespCommand.LPUSH || command == RespCommand.LPUSHX - ? storageApi.ListLeftPush(keyBytes, ref input, out var output) - : storageApi.ListRightPush(keyBytes, ref input, out output); + ? storageApi.ListLeftPush(key, ref input, out var output) + : storageApi.ListRightPush(key, ref input, out output); if (status == GarnetStatus.WRONGTYPE) { @@ -76,8 +76,7 @@ private unsafe bool ListPop(RespCommand command, ref TGarnetApi stor } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var popCount = 1; @@ -105,11 +104,11 @@ private unsafe bool ListPop(RespCommand command, ref TGarnetApi stor var input = new ObjectInput(header, popCount); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var statusOp = command == RespCommand.LPOP - ? storageApi.ListLeftPop(keyBytes, ref input, ref output) - : storageApi.ListRightPop(keyBytes, ref input, ref output); + ? storageApi.ListLeftPop(key, ref input, ref output) + : storageApi.ListRightPop(key, ref input, ref output); switch (statusOp) { @@ -145,17 +144,16 @@ private unsafe bool ListPosition(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LPOS }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var statusOp = storageApi.ListPosition(keyBytes, ref input, ref output); + var statusOp = storageApi.ListPosition(key, ref input, ref output); switch (statusOp) { @@ -163,7 +161,7 @@ private unsafe bool ListPosition(ref TGarnetApi storageApi) ProcessOutput(output.SpanByteAndMemory); break; case GarnetStatus.NOTFOUND: - bool count = false; + var count = false; for (var i = 2; i < parseState.Count; ++i) { if (parseState.GetArgSliceByRef(i).Span.EqualsUpperCaseSpanIgnoringCase(CmdStrings.COUNT)) @@ -220,7 +218,7 @@ private unsafe bool ListPopMultiple(ref TGarnetApi storageApi) } // Get the keys for Lists - var keys = new ArgSlice[numKeys]; + var keys = new PinnedSpanByte[numKeys]; for (var i = 0; i < keys.Length; i++) { @@ -303,7 +301,7 @@ private bool ListBlockingPop(RespCommand command) for (var i = 0; i < keysBytes.Length; i++) { - keysBytes[i] = parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); + keysBytes[i] = parseState.GetArgSliceByRef(i).ToArray(); } if (!parseState.TryGetDouble(parseState.Count - 1, out var timeout)) @@ -380,8 +378,8 @@ private bool ListBlockingPopPush() var srcKey = parseState.GetArgSliceByRef(0); var dstKey = parseState.GetArgSliceByRef(1); - var rightOption = ArgSlice.FromPinnedSpan(CmdStrings.RIGHT); - var leftOption = ArgSlice.FromPinnedSpan(CmdStrings.LEFT); + var rightOption = PinnedSpanByte.FromPinnedSpan(CmdStrings.RIGHT); + var leftOption = PinnedSpanByte.FromPinnedSpan(CmdStrings.LEFT); if (!parseState.TryGetDouble(2, out var timeout)) { @@ -393,9 +391,9 @@ private bool ListBlockingPopPush() return ListBlockingMove(srcKey, dstKey, rightOption, leftOption, timeout); } - private bool ListBlockingMove(ArgSlice srcKey, ArgSlice dstKey, ArgSlice srcDir, ArgSlice dstDir, double timeout) + private bool ListBlockingMove(PinnedSpanByte srcKey, PinnedSpanByte dstKey, PinnedSpanByte srcDir, PinnedSpanByte dstDir, double timeout) { - var cmdArgs = new ArgSlice[] { default, default, default }; + var cmdArgs = new PinnedSpanByte[] { default, default, default }; // Read destination key cmdArgs[0] = dstKey; @@ -410,8 +408,8 @@ private bool ListBlockingMove(ArgSlice srcKey, ArgSlice dstKey, ArgSlice srcDir, var pSrcDir = (byte*)&sourceDirection; var pDstDir = (byte*)&destinationDirection; - cmdArgs[1] = new ArgSlice(pSrcDir, 1); - cmdArgs[2] = new ArgSlice(pDstDir, 1); + cmdArgs[1] = PinnedSpanByte.FromPinnedPointer(pSrcDir, 1); + cmdArgs[2] = PinnedSpanByte.FromPinnedPointer(pDstDir, 1); if (storeWrapper.objectStore == null) throw new GarnetException("Object store is disabled"); @@ -447,14 +445,13 @@ private bool ListLength(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("LLEN"); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LLEN }; var input = new ObjectInput(header); - var status = storageApi.ListLength(keyBytes, ref input, out var output); + var status = storageApi.ListLength(key, ref input, out var output); switch (status) { @@ -492,8 +489,7 @@ private bool ListTrim(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Read the parameters(start and stop) from LTRIM if (!parseState.TryGetInt(1, out var start) || @@ -508,7 +504,7 @@ private bool ListTrim(ref TGarnetApi storageApi) var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LTRIM }; var input = new ObjectInput(header, start, stop); - var status = storageApi.ListTrim(keyBytes, ref input); + var status = storageApi.ListTrim(key, ref input); switch (status) { @@ -543,8 +539,7 @@ private bool ListRange(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Read count start and end params for LRANGE if (!parseState.TryGetInt(1, out var start) || @@ -560,9 +555,9 @@ private bool ListRange(ref TGarnetApi storageApi) var input = new ObjectInput(header, start, end); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var statusOp = storageApi.ListRange(keyBytes, ref input, ref output); + var statusOp = storageApi.ListRange(key, ref input, ref output); switch (statusOp) { @@ -598,8 +593,7 @@ private bool ListIndex(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Read index param if (!parseState.TryGetInt(1, out var index)) @@ -614,9 +608,9 @@ private bool ListIndex(ref TGarnetApi storageApi) var input = new ObjectInput(header, index); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var statusOp = storageApi.ListIndex(keyBytes, ref input, ref output); + var statusOp = storageApi.ListIndex(key, ref input, ref output); switch (statusOp) { @@ -654,14 +648,13 @@ private bool ListInsert(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LINSERT }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var statusOp = storageApi.ListInsert(keyBytes, ref input, out var output); + var statusOp = storageApi.ListInsert(key, ref input, out var output); switch (statusOp) { @@ -702,8 +695,7 @@ private bool ListRemove(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Get count parameter if (!parseState.TryGetInt(1, out var nCount)) @@ -717,7 +709,7 @@ private bool ListRemove(ref TGarnetApi storageApi) var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LREM }; var input = new ObjectInput(header, ref parseState, startIdx: 2, arg1: nCount); - var statusOp = storageApi.ListRemove(keyBytes, ref input, out var output); + var statusOp = storageApi.ListRemove(key, ref input, out var output); switch (statusOp) { @@ -852,16 +844,12 @@ private bool ListRightPopLeftPush(ref TGarnetApi storageApi) /// /// /// - private bool ListMove(ArgSlice sourceKey, ArgSlice destinationKey, + private bool ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] node, ref TGarnetApi storageApi, out GarnetStatus garnetStatus) where TGarnetApi : IGarnetApi { - garnetStatus = GarnetStatus.OK; - node = null; - - garnetStatus = - storageApi.ListMove(sourceKey, destinationKey, sourceDirection, destinationDirection, out node); + garnetStatus = storageApi.ListMove(sourceKey, destinationKey, sourceDirection, destinationDirection, out node); return true; } @@ -881,17 +869,16 @@ public bool ListSet(ref TGarnetApi storageApi) } // Get the key for List - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LSET }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var statusOp = storageApi.ListSet(keyBytes, ref input, ref output); + var statusOp = storageApi.ListSet(key, ref input, ref output); switch (statusOp) { @@ -946,21 +933,18 @@ private unsafe bool ListBlockingPopMultiple() // Get the keys for Lists var keysBytes = new byte[numKeys][]; for (var i = 0; i < keysBytes.Length; i++) - { - keysBytes[i] = parseState.GetArgSliceByRef(currTokenId++).SpanByte.ToByteArray(); - } + keysBytes[i] = parseState.GetArgSliceByRef(currTokenId++).ToArray(); - var cmdArgs = new ArgSlice[2]; + var cmdArgs = new PinnedSpanByte[2]; // Get the direction var dir = parseState.GetArgSliceByRef(currTokenId++); var popDirection = GetOperationDirection(dir); if (popDirection == OperationDirection.Unknown) - { return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); - } - cmdArgs[0] = new ArgSlice((byte*)&popDirection, 1); + + cmdArgs[0] = PinnedSpanByte.FromPinnedPointer((byte*)&popDirection, 1); var popCount = 1; @@ -982,7 +966,7 @@ private unsafe bool ListBlockingPopMultiple() } } - cmdArgs[1] = new ArgSlice((byte*)&popCount, sizeof(int)); + cmdArgs[1] = PinnedSpanByte.FromPinnedPointer((byte*)&popCount, sizeof(int)); if (storeWrapper.objectStore == null) throw new GarnetException("Object store is disabled"); diff --git a/libs/server/Resp/Objects/ObjectStoreUtils.cs b/libs/server/Resp/Objects/ObjectStoreUtils.cs index 4264b1c1892..5d5a98ff798 100644 --- a/libs/server/Resp/Objects/ObjectStoreUtils.cs +++ b/libs/server/Resp/Objects/ObjectStoreUtils.cs @@ -4,6 +4,7 @@ using System; using System.Text; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -92,7 +93,7 @@ private bool AbortWithErrorMessage(string format, params object[] args) /// /// The input to parse. /// The parsed OperationDirection, or OperationDirection.Unknown if parsing fails. - public OperationDirection GetOperationDirection(ArgSlice input) + public OperationDirection GetOperationDirection(PinnedSpanByte input) { // Optimize for the common case if (input.ReadOnlySpan.SequenceEqual("LEFT"u8)) @@ -100,7 +101,7 @@ public OperationDirection GetOperationDirection(ArgSlice input) if (input.ReadOnlySpan.SequenceEqual("RIGHT"u8)) return OperationDirection.Right; // Rare case: try making upper case and retry - MakeUpperCase(input.ptr); + MakeUpperCase(input.ToPointer()); if (input.ReadOnlySpan.SequenceEqual("LEFT"u8)) return OperationDirection.Left; if (input.ReadOnlySpan.SequenceEqual("RIGHT"u8)) diff --git a/libs/server/Resp/Objects/SetCommands.cs b/libs/server/Resp/Objects/SetCommands.cs index 7d7438ec465..e84148de68a 100644 --- a/libs/server/Resp/Objects/SetCommands.cs +++ b/libs/server/Resp/Objects/SetCommands.cs @@ -5,6 +5,7 @@ using System.Diagnostics; using System.Text; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -30,14 +31,13 @@ private unsafe bool SetAdd(ref TGarnetApi storageApi) } // Get the key for the Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SADD }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.SetAdd(keyBytes, ref input, out var output); + var status = storageApi.SetAdd(key, ref input, out var output); switch (status) { @@ -71,7 +71,7 @@ private bool SetIntersect(ref TGarnetApi storageApi) } // Read all keys - var keys = new ArgSlice[parseState.Count]; + var keys = new PinnedSpanByte[parseState.Count]; for (var i = 0; i < keys.Length; i++) { keys[i] = parseState.GetArgSliceByRef(i); @@ -125,15 +125,15 @@ private bool SetIntersectStore(ref TGarnetApi storageApi) } // Get the key - var keyBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); - var keys = new ArgSlice[parseState.Count - 1]; + var keys = new PinnedSpanByte[parseState.Count - 1]; for (var i = 1; i < parseState.Count; i++) { keys[i - 1] = parseState.GetArgSliceByRef(i); } - var status = storageApi.SetIntersectStore(keyBytes, keys, out var output); + var status = storageApi.SetIntersectStore(key, keys, out var output); switch (status) { case GarnetStatus.OK: @@ -231,7 +231,7 @@ private bool SetUnion(ref TGarnetApi storageApi) } // Read all the keys - var keys = new ArgSlice[parseState.Count]; + var keys = new PinnedSpanByte[parseState.Count]; for (var i = 0; i < keys.Length; i++) { @@ -276,15 +276,15 @@ private bool SetUnionStore(ref TGarnetApi storageApi) } // Get the key - var keyBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); - var keys = new ArgSlice[parseState.Count - 1]; + var keys = new PinnedSpanByte[parseState.Count - 1]; for (var i = 1; i < parseState.Count; i++) { keys[i - 1] = parseState.GetArgSliceByRef(i); } - var status = storageApi.SetUnionStore(keyBytes, keys, out var output); + var status = storageApi.SetUnionStore(key, keys, out var output); switch (status) { case GarnetStatus.OK: @@ -316,15 +316,14 @@ private unsafe bool SetRemove(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("SREM"); } - // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + // Get the key + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SREM }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.SetRemove(keyBytes, ref input, out var output); + var status = storageApi.SetRemove(key, ref input, out var output); switch (status) { @@ -361,14 +360,13 @@ private unsafe bool SetLength(ref TGarnetApi storageApi) } // Get the key for the Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SCARD }; var input = new ObjectInput(header); - var status = storageApi.SetLength(keyBytes, ref input, out var output); + var status = storageApi.SetLength(key, ref input, out var output); switch (status) { @@ -405,17 +403,16 @@ private unsafe bool SetMembers(ref TGarnetApi storageApi) } // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SMEMBERS }; var input = new ObjectInput(header); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SetMembers(keyBytes, ref input, ref output); + var status = storageApi.SetMembers(key, ref input, ref output); switch (status) { @@ -457,17 +454,16 @@ private unsafe bool SetIsMember(RespCommand cmd, ref TGarnetApi stor } // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = isSingle ? SetOperation.SISMEMBER : SetOperation.SMISMEMBER }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SetIsMember(keyBytes, ref input, ref output); + var status = storageApi.SetIsMember(key, ref input, ref output); switch (status) { @@ -518,9 +514,7 @@ private unsafe bool SetPop(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("SPOP"); } - // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var countParameter = int.MinValue; if (parseState.Count == 2) @@ -545,9 +539,9 @@ private unsafe bool SetPop(ref TGarnetApi storageApi) var input = new ObjectInput(header, countParameter); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SetPop(keyBytes, ref input, ref output); + var status = storageApi.SetPop(key, ref input, ref output); switch (status) { @@ -630,9 +624,7 @@ private unsafe bool SetRandomMember(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("SRANDMEMBER"); } - // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var countParameter = int.MinValue; if (parseState.Count == 2) @@ -660,9 +652,9 @@ private unsafe bool SetRandomMember(ref TGarnetApi storageApi) var input = new ObjectInput(header, countParameter, seed); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SetRandomMember(keyBytes, ref input, ref output); + var status = storageApi.SetRandomMember(key, ref input, ref output); switch (status) { @@ -703,7 +695,7 @@ private bool SetDiff(ref TGarnetApi storageApi) return AbortWithWrongNumberOfArguments("SDIFF"); } - var keys = new ArgSlice[parseState.Count]; + var keys = new PinnedSpanByte[parseState.Count]; for (var i = 0; i < parseState.Count; i++) { keys[i] = parseState.GetArgSliceByRef(i); @@ -746,15 +738,15 @@ private bool SetDiffStore(ref TGarnetApi storageApi) } // Get the key - var keyBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); - var keys = new ArgSlice[parseState.Count - 1]; + var keys = new PinnedSpanByte[parseState.Count - 1]; for (var i = 1; i < parseState.Count; i++) { keys[i - 1] = parseState.GetArgSliceByRef(i); } - var status = storageApi.SetDiffStore(keyBytes, keys, out var output); + var status = storageApi.SetDiffStore(key, keys, out var output); switch (status) { case GarnetStatus.OK: diff --git a/libs/server/Resp/Objects/SharedObjectCommands.cs b/libs/server/Resp/Objects/SharedObjectCommands.cs index 4be64d4ec21..9002c1ab16e 100644 --- a/libs/server/Resp/Objects/SharedObjectCommands.cs +++ b/libs/server/Resp/Objects/SharedObjectCommands.cs @@ -34,8 +34,7 @@ private unsafe bool ObjectScan(GarnetObjectType objectType, ref TGar } // Read key for the scan - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Get cursor value if (!parseState.TryGetLong(1, out var cursorValue) || cursorValue < 0) @@ -66,8 +65,8 @@ private unsafe bool ObjectScan(GarnetObjectType objectType, ref TGar } // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); - var status = storageApi.ObjectScan(keyBytes, ref input, ref output); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + var status = storageApi.ObjectScan(key, ref input, ref output); switch (status) { diff --git a/libs/server/Resp/Objects/SortedSetCommands.cs b/libs/server/Resp/Objects/SortedSetCommands.cs index f5592776508..53e7820189a 100644 --- a/libs/server/Resp/Objects/SortedSetCommands.cs +++ b/libs/server/Resp/Objects/SortedSetCommands.cs @@ -23,20 +23,17 @@ private unsafe bool SortedSetAdd(ref TGarnetApi storageApi) where TGarnetApi : IGarnetApi { if (parseState.Count < 3) - { return AbortWithWrongNumberOfArguments("ZADD"); - } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZADD }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetAdd(keyBytes, ref input, ref output); + var status = storageApi.SortedSetAdd(key, ref input, ref output); switch (status) { @@ -68,13 +65,12 @@ private unsafe bool SortedSetRemove(ref TGarnetApi storageApi) } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZREM }; var input = new ObjectInput(header, ref parseState, startIdx: 1); - var status = storageApi.SortedSetRemove(keyBytes, ref input, out var rmwOutput); + var status = storageApi.SortedSetRemove(key, ref input, out var rmwOutput); switch (status) { @@ -109,13 +105,12 @@ private unsafe bool SortedSetLength(ref TGarnetApi storageApi) } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZCARD }; var input = new ObjectInput(header); - var status = storageApi.SortedSetLength(keyBytes, ref input, out var output); + var status = storageApi.SortedSetLength(key, ref input, out var output); switch (status) { @@ -155,8 +150,7 @@ private unsafe bool SortedSetRange(RespCommand command, ref TGarnetA return AbortWithWrongNumberOfArguments(nameof(RespCommand.ZRANGE)); } - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var rangeOpts = SortedSetRangeOpts.None; @@ -187,9 +181,9 @@ private unsafe bool SortedSetRange(RespCommand command, ref TGarnetA var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZRANGE }; var input = new ObjectInput(header, ref parseState, startIdx: 1, arg1: respProtocolVersion, arg2: (int)rangeOpts); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetRange(keyBytes, ref input, ref output); + var status = storageApi.SortedSetRange(key, ref input, ref output); switch (status) { @@ -258,17 +252,16 @@ private unsafe bool SortedSetScore(ref TGarnetApi storageApi) } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZSCORE }; var input = new ObjectInput(header, ref parseState, startIdx: 1, arg1: respProtocolVersion); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetScore(keyBytes, ref input, ref output); + var status = storageApi.SortedSetScore(key, ref input, ref output); switch (status) { @@ -304,17 +297,16 @@ private unsafe bool SortedSetScores(ref TGarnetApi storageApi) } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZMSCORE }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetScores(keyBytes, ref input, ref output); + var status = storageApi.SortedSetScores(key, ref input, ref output); switch (status) { @@ -355,8 +347,7 @@ private unsafe bool SortedSetPop(RespCommand command, ref TGarnetApi } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var popCount = -1; @@ -382,9 +373,9 @@ private unsafe bool SortedSetPop(RespCommand command, ref TGarnetApi var input = new ObjectInput(header, popCount); // Prepare output - var output = new GarnetObjectStoreOutput(new SpanByteAndMemory(SpanByte.FromPinnedPointer(dcurr, (int)(dend - dcurr)))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetPop(keyBytes, ref input, ref output); + var status = storageApi.SortedSetPop(key, ref input, ref output); switch (status) { @@ -534,17 +525,16 @@ private unsafe bool SortedSetCount(ref TGarnetApi storageApi) } // Get the key for the Sorted Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZCOUNT }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare output - var output = new GarnetObjectStoreOutput(new SpanByteAndMemory(SpanByte.FromPinnedPointer(dcurr, (int)(dend - dcurr)))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetCount(keyBytes, ref input, ref output); + var status = storageApi.SortedSetCount(key, ref input, ref output); switch (status) { @@ -584,8 +574,7 @@ private unsafe bool SortedSetLengthByValue(RespCommand command, ref } // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var op = command switch @@ -600,8 +589,8 @@ private unsafe bool SortedSetLengthByValue(RespCommand command, ref var input = new ObjectInput(header, ref parseState, startIdx: 1); var status = op == SortedSetOperation.ZREMRANGEBYLEX ? - storageApi.SortedSetRemoveRangeByLex(keyBytes, ref input, out var output) : - storageApi.SortedSetLengthByValue(keyBytes, ref input, out output); + storageApi.SortedSetRemoveRangeByLex(key, ref input, out var output) : + storageApi.SortedSetLengthByValue(key, ref input, out output); switch (status) { @@ -649,17 +638,16 @@ private unsafe bool SortedSetIncrement(ref TGarnetApi storageApi) } // Get the key for the Sorted Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); // Prepare input var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZINCRBY }; var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetIncrement(keyBytes, ref input, ref output); + var status = storageApi.SortedSetIncrement(key, ref input, ref output); switch (status) { @@ -693,8 +681,7 @@ private unsafe bool SortedSetRank(RespCommand command, ref TGarnetAp } // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var includeWithScore = false; // Read WITHSCORE @@ -723,9 +710,9 @@ private unsafe bool SortedSetRank(RespCommand command, ref TGarnetAp var input = new ObjectInput(header, ref parseState, startIdx: 1, arg1: includeWithScore ? 1 : 0); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetRank(keyBytes, ref input, ref output); + var status = storageApi.SortedSetRank(key, ref input, ref output); switch (status) { @@ -762,8 +749,7 @@ private unsafe bool SortedSetRemoveRange(RespCommand command, ref TG } // Get the key - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var op = command switch @@ -778,9 +764,9 @@ private unsafe bool SortedSetRemoveRange(RespCommand command, ref TG var input = new ObjectInput(header, ref parseState, startIdx: 1); // Prepare GarnetObjectStore output - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.SortedSetRemoveRange(keyBytes, ref input, ref output); + var status = storageApi.SortedSetRemoveRange(key, ref input, ref output); switch (status) { @@ -814,8 +800,7 @@ private unsafe bool SortedSetRandomMember(ref TGarnetApi storageApi) } // Get the key for the Sorted Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); var paramCount = 1; var includeWithScores = false; var includedCount = false; @@ -862,8 +847,8 @@ private unsafe bool SortedSetRandomMember(ref TGarnetApi storageApi) if (paramCount != 0) { // Prepare GarnetObjectStore output - output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); - status = storageApi.SortedSetRandomMember(keyBytes, ref input, ref output); + output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); + status = storageApi.SortedSetRandomMember(key, ref input, ref output); } switch (status) @@ -929,7 +914,7 @@ private unsafe bool SortedSetDifference(ref TGarnetApi storageApi) return true; } - var keys = new ArgSlice[nKeys]; + var keys = new PinnedSpanByte[nKeys]; for (var i = 1; i < nKeys + 1; i++) { @@ -1559,9 +1544,7 @@ private unsafe bool SortedSetBlockingPop(RespCommand command) var keysBytes = new byte[parseState.Count - 1][]; for (var i = 0; i < keysBytes.Length; i++) - { - keysBytes[i] = parseState.GetArgSliceByRef(i).SpanByte.ToByteArray(); - } + keysBytes[i] = parseState.GetArgSliceByRef(i).ToArray(); var result = storeWrapper.itemBroker.GetCollectionItemAsync(command, keysBytes, this, timeout).Result; @@ -1626,11 +1609,9 @@ private unsafe bool SortedSetBlockingMPop() var keysBytes = new byte[numKeys][]; for (var i = 0; i < keysBytes.Length; i++) - { - keysBytes[i] = parseState.GetArgSliceByRef(currTokenId++).SpanByte.ToByteArray(); - } + keysBytes[i] = parseState.GetArgSliceByRef(currTokenId++).ToArray(); - var cmdArgs = new ArgSlice[2]; + var cmdArgs = new PinnedSpanByte[2]; var orderArg = parseState.GetArgSliceByRef(currTokenId++); var orderSpan = orderArg.ReadOnlySpan; @@ -1645,7 +1626,7 @@ private unsafe bool SortedSetBlockingMPop() return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); } - cmdArgs[0] = new ArgSlice((byte*)&lowScoresFirst, 1); + cmdArgs[0] = PinnedSpanByte.FromPinnedPointer((byte*)&lowScoresFirst, 1); var popCount = 1; @@ -1664,7 +1645,7 @@ private unsafe bool SortedSetBlockingMPop() } } - cmdArgs[1] = new ArgSlice((byte*)&popCount, sizeof(int)); + cmdArgs[1] = PinnedSpanByte.FromPinnedPointer((byte*)&popCount, sizeof(int)); var result = storeWrapper.itemBroker.GetCollectionItemAsync(RespCommand.BZMPOP, keysBytes, this, timeout, cmdArgs).Result; @@ -1773,7 +1754,7 @@ private unsafe bool SortedSetExpire(RespCommand command, ref TGarnet var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZEXPIRE }; var input = new ObjectInput(header, ref parseState, startIdx: 1, (int)expireOption, (int)inputFlag); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.SortedSetExpire(key, ref input, ref output); @@ -1866,7 +1847,7 @@ private unsafe bool SortedSetTimeToLive(RespCommand command, ref TGa var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZTTL }; var input = new ObjectInput(header, ref membersParseState, arg1: isMilliseconds ? 1 : 0, arg2: isTimestamp ? 1 : 0); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.SortedSetTimeToLive(key, ref input, ref output); @@ -1935,7 +1916,7 @@ private unsafe bool SortedSetPersist(ref TGarnetApi storageApi) var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZPERSIST }; var input = new ObjectInput(header, ref membersParseState); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); var status = storageApi.SortedSetPersist(key, ref input, ref output); diff --git a/libs/server/Resp/Objects/SortedSetGeoCommands.cs b/libs/server/Resp/Objects/SortedSetGeoCommands.cs index d88ddfe0813..df30554f8b5 100644 --- a/libs/server/Resp/Objects/SortedSetGeoCommands.cs +++ b/libs/server/Resp/Objects/SortedSetGeoCommands.cs @@ -29,7 +29,7 @@ private unsafe bool GeoAdd(ref TGarnetApi storageApi) var currTokenIdx = 0; // Get the key for SortedSet - var sbKey = parseState.GetArgSliceByRef(currTokenIdx++).SpanByte; + var key = parseState.GetArgSliceByRef(currTokenIdx++); while (currTokenIdx < parseState.Count) { @@ -85,9 +85,9 @@ private unsafe bool GeoAdd(ref TGarnetApi storageApi) var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.GEOADD }; var input = new ObjectInput(header, ref parseState, startIdx: memberStart, arg1: (int)addOption); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.GeoAdd(sbKey.ToByteArray(), ref input, ref output); + var status = storageApi.GeoAdd(key, ref input, ref output); switch (status) { @@ -137,8 +137,7 @@ private unsafe bool GeoCommands(RespCommand command, ref TGarnetApi } // Get the key for the Sorted Set - var sbKey = parseState.GetArgSliceByRef(0).SpanByte; - var keyBytes = sbKey.ToByteArray(); + var key = parseState.GetArgSliceByRef(0); SortedSetOperation op; @@ -166,9 +165,9 @@ private unsafe bool GeoCommands(RespCommand command, ref TGarnetApi var input = new ObjectInput(header, ref parseState, startIdx: 1); - var output = new GarnetObjectStoreOutput(new(dcurr, (int)(dend - dcurr))); + var output = GarnetObjectStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr)); - var status = storageApi.GeoCommands(keyBytes, ref input, ref output); + var status = storageApi.GeoCommands(key, ref input, ref output); switch (status) { @@ -259,7 +258,7 @@ private unsafe bool GeoSearchCommands(RespCommand command, ref TGarn { SortedSetOp = SortedSetOperation.GEOSEARCH }, ref parseState, startIdx: sourceIdx + 1, arg1: (int)command); - var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var output = SpanByteAndMemory.FromPinnedPointer(dcurr, (int)(dend - dcurr)); if (!input.parseState.TryGetGeoSearchOptions(command, out var searchOpts, out var destIdx, out var errorMessage)) { diff --git a/libs/server/Resp/Parser/ParseUtils.cs b/libs/server/Resp/Parser/ParseUtils.cs index 6fd9130b2df..13c580468b1 100644 --- a/libs/server/Resp/Parser/ParseUtils.cs +++ b/libs/server/Resp/Parser/ParseUtils.cs @@ -6,6 +6,7 @@ using System.Text; using Garnet.common; using Garnet.common.Parsing; +using Tsavorite.core; namespace Garnet.server { @@ -21,7 +22,7 @@ public static unsafe class ParseUtils /// Parsed integer /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int ReadInt(ref ArgSlice slice) + public static int ReadInt(PinnedSpanByte slice) { int number = default; var ptr = slice.ptr; @@ -43,7 +44,7 @@ public static int ReadInt(ref ArgSlice slice) /// True if integer read successfully /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool TryReadInt(ref ArgSlice slice, out int number) + public static bool TryReadInt(PinnedSpanByte slice, out int number) { number = default; var ptr = slice.ptr; @@ -60,7 +61,7 @@ public static bool TryReadInt(ref ArgSlice slice, out int number) /// Parsed long /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static long ReadLong(ref ArgSlice slice) + public static long ReadLong(PinnedSpanByte slice) { long number = default; var ptr = slice.ptr; @@ -82,7 +83,7 @@ public static long ReadLong(ref ArgSlice slice) /// True if long parsed successfully /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool TryReadLong(ref ArgSlice slice, out long number) + public static bool TryReadLong(PinnedSpanByte slice, out long number) { number = default; var ptr = slice.ptr; @@ -99,9 +100,9 @@ public static bool TryReadLong(ref ArgSlice slice, out long number) /// Parsed double /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static double ReadDouble(ref ArgSlice slice) + public static double ReadDouble(PinnedSpanByte slice) { - if (!TryReadDouble(ref slice, out var number)) + if (!TryReadDouble(slice, out var number)) { RespParsingException.ThrowNotANumber(slice.ptr, slice.length); } @@ -115,7 +116,7 @@ public static double ReadDouble(ref ArgSlice slice) /// True if double parsed successfully /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool TryReadDouble(ref ArgSlice slice, out double number) + public static bool TryReadDouble(PinnedSpanByte slice, out double number) { var sbNumber = slice.ReadOnlySpan; return Utf8Parser.TryParse(sbNumber, out number, out var bytesConsumed) && @@ -129,7 +130,7 @@ public static bool TryReadDouble(ref ArgSlice slice, out double number) /// Parsed string /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static string ReadString(ref ArgSlice slice) + public static string ReadString(PinnedSpanByte slice) { return Encoding.ASCII.GetString(slice.ReadOnlySpan); } @@ -141,9 +142,9 @@ public static string ReadString(ref ArgSlice slice) /// Parsed integer /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool ReadBool(ref ArgSlice slice) + public static bool ReadBool(PinnedSpanByte slice) { - if (!TryReadBool(ref slice, out var value)) + if (!TryReadBool(slice, out var value)) { RespParsingException.ThrowNotANumber(slice.ptr, slice.length); } @@ -157,7 +158,7 @@ public static bool ReadBool(ref ArgSlice slice) /// True if integer read successfully /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool TryReadBool(ref ArgSlice slice, out bool value) + public static bool TryReadBool(PinnedSpanByte slice, out bool value) { value = false; diff --git a/libs/server/Resp/Parser/SessionParseState.cs b/libs/server/Resp/Parser/SessionParseState.cs index 963f9b15cba..36d3359d1c7 100644 --- a/libs/server/Resp/Parser/SessionParseState.cs +++ b/libs/server/Resp/Parser/SessionParseState.cs @@ -27,9 +27,9 @@ public unsafe struct SessionParseState public int Count; /// - /// Pointer to accessible buffer + /// Pointer to the slice of (which is always pinned) that is accessible within the range of this instance's arguments. /// - ArgSlice* bufferPtr; + PinnedSpanByte* bufferPtr; /// /// Count of arguments in the original buffer @@ -37,16 +37,16 @@ public unsafe struct SessionParseState int rootCount; /// - /// Arguments original buffer + /// Arguments original buffer (always pinned) /// - ArgSlice[] rootBuffer; + PinnedSpanByte[] rootBuffer; /// /// Get a Span of the parsed parameters in the form an ArgSlice /// - public ReadOnlySpan Parameters => new(bufferPtr, Count); + public ReadOnlySpan Parameters => new(bufferPtr, Count); - private SessionParseState(ref ArgSlice[] rootBuffer, int rootCount, ref ArgSlice* bufferPtr, int count) : this() + private SessionParseState(ref PinnedSpanByte[] rootBuffer, int rootCount, ref PinnedSpanByte* bufferPtr, int count) : this() { this.rootBuffer = rootBuffer; this.rootCount = rootCount; @@ -61,8 +61,8 @@ public void Initialize() { Count = 0; rootCount = 0; - rootBuffer = GC.AllocateArray(MinParams, true); - bufferPtr = (ArgSlice*)Unsafe.AsPointer(ref rootBuffer[0]); + rootBuffer = GC.AllocateArray(MinParams, true); + bufferPtr = (PinnedSpanByte*)Unsafe.AsPointer(ref rootBuffer[0]); } /// @@ -78,8 +78,8 @@ public void Initialize(int count) if (rootBuffer != null && (count <= MinParams || count <= rootBuffer.Length)) return; - rootBuffer = GC.AllocateArray(count <= MinParams ? MinParams : count, true); - bufferPtr = (ArgSlice*)Unsafe.AsPointer(ref rootBuffer[0]); + rootBuffer = GC.AllocateArray(count <= MinParams ? MinParams : count, true); + bufferPtr = (PinnedSpanByte*)Unsafe.AsPointer(ref rootBuffer[0]); } /// @@ -87,7 +87,7 @@ public void Initialize(int count) /// /// Argument to initialize buffer with [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArgument(ArgSlice arg) + public void InitializeWithArgument(PinnedSpanByte arg) { Initialize(1); @@ -100,7 +100,7 @@ public void InitializeWithArgument(ArgSlice arg) /// First argument /// Second argument [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2) + public void InitializeWithArguments(PinnedSpanByte arg1, PinnedSpanByte arg2) { Initialize(2); @@ -115,7 +115,7 @@ public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2) /// Second argument /// Third argument [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3) + public void InitializeWithArguments(PinnedSpanByte arg1, PinnedSpanByte arg2, PinnedSpanByte arg3) { Initialize(3); @@ -132,7 +132,7 @@ public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3) /// Third argument /// Fourth argument [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3, ArgSlice arg4) + public void InitializeWithArguments(PinnedSpanByte arg1, PinnedSpanByte arg2, PinnedSpanByte arg3, PinnedSpanByte arg4) { Initialize(4); @@ -151,7 +151,7 @@ public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3, /// Fourth argument /// Fifth argument [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3, ArgSlice arg4, ArgSlice arg5) + public void InitializeWithArguments(PinnedSpanByte arg1, PinnedSpanByte arg2, PinnedSpanByte arg3, PinnedSpanByte arg4, PinnedSpanByte arg5) { Initialize(5); @@ -167,7 +167,7 @@ public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3, /// /// Set of arguments to initialize buffer with [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ArgSlice[] args) + public void InitializeWithArguments(PinnedSpanByte[] args) { Initialize(args.Length); @@ -209,7 +209,7 @@ public SessionParseState Slice(int idxOffset, int count) /// /// Set of arguments to initialize buffer with [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeWithArguments(ReadOnlySpan args) + public void InitializeWithArguments(ReadOnlySpan args) { Initialize(args.Length); @@ -224,7 +224,7 @@ public void InitializeWithArguments(ReadOnlySpan args) /// /// Index of buffer at which to set argument /// Argument to set - public void SetArgument(int i, ArgSlice arg) + public void SetArgument(int i, PinnedSpanByte arg) { Debug.Assert(i < Count); *(bufferPtr + i) = arg; @@ -235,13 +235,11 @@ public void SetArgument(int i, ArgSlice arg) /// /// Index of buffer at which to start setting arguments /// Arguments to set - public void SetArguments(int i, params ArgSlice[] args) + public void SetArguments(int i, params PinnedSpanByte[] args) { Debug.Assert(i + args.Length - 1 < Count); for (var j = 0; j < args.Length; j++) - { *(bufferPtr + i + j) = args[j]; - } } /// @@ -249,13 +247,11 @@ public void SetArguments(int i, params ArgSlice[] args) /// /// Index of buffer at which to start setting arguments /// Arguments to set - public void SetArguments(int i, params ReadOnlySpan args) + public void SetArguments(int i, params ReadOnlySpan args) { Debug.Assert(i + args.Length - 1 < Count); for (var j = 0; j < args.Length; j++) - { *(bufferPtr + i + j) = args[j]; - } } /// @@ -267,10 +263,7 @@ public int GetSerializedLength() var serializedLength = sizeof(int); for (var i = 0; i < Count; i++) - { - serializedLength += (*(bufferPtr + i)).SpanByte.TotalSize; - } - + serializedLength += (*(bufferPtr + i)).TotalSize; return serializedLength; } @@ -281,7 +274,7 @@ public int GetSerializedLength() /// The memory buffer to serialize into (of size at least SerializedLength(firstIdx) bytes) /// Length of buffer to serialize into. /// Total serialized bytes - public int CopyTo(byte* dest, int length) + public int SerializeTo(byte* dest, int length) { var curr = dest; @@ -292,8 +285,8 @@ public int CopyTo(byte* dest, int length) // Serialize arguments for (var i = 0; i < Count; i++) { - var sbParam = (*(bufferPtr + i)).SpanByte; - sbParam.CopyTo(curr); + var sbParam = *(bufferPtr + i); + sbParam.SerializeTo(curr); curr += sbParam.TotalSize; } @@ -316,9 +309,9 @@ public unsafe int DeserializeFrom(byte* src) for (var i = 0; i < argCount; i++) { - ref var sbArgument = ref Unsafe.AsRef(curr); - *(bufferPtr + i) = new ArgSlice(ref sbArgument); - curr += sbArgument.TotalSize; + var argument = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr); + *(bufferPtr + i) = argument; + curr += argument.TotalSize; } return (int)(src - curr); @@ -331,27 +324,20 @@ public unsafe int DeserializeFrom(byte* src) public bool Read(int i, ref byte* ptr, byte* end) { Debug.Assert(i < Count); - ref var slice = ref Unsafe.AsRef(bufferPtr + i); + ref var slice = ref Unsafe.AsRef(bufferPtr + i); // Parse RESP string header - if (!RespReadUtils.TryReadUnsignedLengthHeader(out slice.length, ref ptr, end)) - { + if (!RespReadUtils.TryReadUnsignedLengthHeader(out var length, ref ptr, end)) return false; - } - - slice.ptr = ptr; + slice.Set(ptr, length); // Parse content: ensure that input contains key + '\r\n' - ptr += slice.length + 2; + ptr += slice.Length + 2; if (ptr > end) - { return false; - } if (*(ushort*)(ptr - 2) != MemoryMarshal.Read("\r\n"u8)) - { RespParsingException.ThrowUnexpectedToken(*(ptr - 2)); - } return true; } @@ -360,10 +346,10 @@ public bool Read(int i, ref byte* ptr, byte* end) /// Get the argument at the given index /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref ArgSlice GetArgSliceByRef(int i) + public ref PinnedSpanByte GetArgSliceByRef(int i) { Debug.Assert(i < Count); - return ref Unsafe.AsRef(bufferPtr + i); + return ref Unsafe.AsRef(bufferPtr + i); } /// @@ -374,7 +360,7 @@ public ref ArgSlice GetArgSliceByRef(int i) public int GetInt(int i) { Debug.Assert(i < Count); - return ParseUtils.ReadInt(ref Unsafe.AsRef(bufferPtr + i)); + return ParseUtils.ReadInt(*(bufferPtr + i)); } /// @@ -385,7 +371,7 @@ public int GetInt(int i) public bool TryGetInt(int i, out int value) { Debug.Assert(i < Count); - return ParseUtils.TryReadInt(ref Unsafe.AsRef(bufferPtr + i), out value); + return ParseUtils.TryReadInt(*(bufferPtr + i), out value); } /// @@ -396,7 +382,7 @@ public bool TryGetInt(int i, out int value) public long GetLong(int i) { Debug.Assert(i < Count); - return ParseUtils.ReadLong(ref Unsafe.AsRef(bufferPtr + i)); + return ParseUtils.ReadLong(*(bufferPtr + i)); } /// @@ -407,7 +393,7 @@ public long GetLong(int i) public bool TryGetLong(int i, out long value) { Debug.Assert(i < Count); - return ParseUtils.TryReadLong(ref Unsafe.AsRef(bufferPtr + i), out value); + return ParseUtils.TryReadLong(*(bufferPtr + i), out value); } /// @@ -418,7 +404,7 @@ public bool TryGetLong(int i, out long value) public double GetDouble(int i) { Debug.Assert(i < Count); - return ParseUtils.ReadDouble(ref Unsafe.AsRef(bufferPtr + i)); + return ParseUtils.ReadDouble(*(bufferPtr + i)); } /// @@ -429,7 +415,7 @@ public double GetDouble(int i) public bool TryGetDouble(int i, out double value) { Debug.Assert(i < Count); - return ParseUtils.TryReadDouble(ref Unsafe.AsRef(bufferPtr + i), out value); + return ParseUtils.TryReadDouble(*(bufferPtr + i), out value); } /// @@ -440,7 +426,7 @@ public bool TryGetDouble(int i, out double value) public string GetString(int i) { Debug.Assert(i < Count); - return ParseUtils.ReadString(ref Unsafe.AsRef(bufferPtr + i)); + return ParseUtils.ReadString(*(bufferPtr + i)); } /// @@ -451,7 +437,7 @@ public string GetString(int i) public bool GetBool(int i) { Debug.Assert(i < Count); - return ParseUtils.ReadBool(ref Unsafe.AsRef(bufferPtr + i)); + return ParseUtils.ReadBool(*(bufferPtr + i)); } /// @@ -462,7 +448,7 @@ public bool GetBool(int i) public bool TryGetBool(int i, out bool value) { Debug.Assert(i < Count); - return ParseUtils.TryReadBool(ref Unsafe.AsRef(bufferPtr + i), out value); + return ParseUtils.TryReadBool(*(bufferPtr + i), out value); } } } \ No newline at end of file diff --git a/libs/server/Resp/PubSubCommands.cs b/libs/server/Resp/PubSubCommands.cs index 277c03fb972..c28cc7f8a5e 100644 --- a/libs/server/Resp/PubSubCommands.cs +++ b/libs/server/Resp/PubSubCommands.cs @@ -5,6 +5,7 @@ using System.Collections.Generic; using System.Diagnostics; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -18,7 +19,7 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase int numActiveChannels = 0; /// - public override unsafe void Publish(ArgSlice key, ArgSlice value) + public override unsafe void Publish(PinnedSpanByte key, PinnedSpanByte value) { try { @@ -48,7 +49,7 @@ public override unsafe void Publish(ArgSlice key, ArgSlice value) } /// - public override unsafe void PatternPublish(ArgSlice pattern, ArgSlice key, ArgSlice value) + public override unsafe void PatternPublish(PinnedSpanByte pattern, PinnedSpanByte key, PinnedSpanByte value) { try { diff --git a/libs/server/Resp/RespCommandAccessor.cs b/libs/server/Resp/RespCommandAccessor.cs index b5a3ac927fb..a9310196213 100644 --- a/libs/server/Resp/RespCommandAccessor.cs +++ b/libs/server/Resp/RespCommandAccessor.cs @@ -4,7 +4,7 @@ namespace Garnet.server { /// - /// RESP command accessor + /// Accessor to simplify access for a subset of usable by external components such as cluster plugins. /// public static class RespCommandAccessor { diff --git a/libs/server/Resp/RespCommandKeySpecification.cs b/libs/server/Resp/RespCommandKeySpecification.cs index 7cbc2f49452..23183827ea5 100644 --- a/libs/server/Resp/RespCommandKeySpecification.cs +++ b/libs/server/Resp/RespCommandKeySpecification.cs @@ -8,6 +8,7 @@ using System.Text.Json; using System.Text.Json.Serialization; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -328,7 +329,7 @@ public abstract class FindKeysKeySpecMethodBase : KeySpecMethodBase /// The current session parse state. /// The index from which to start extracting keys. /// The list to which extracted keys will be added. - public abstract void ExtractKeys(ref SessionParseState state, int startIndex, List keys); + public abstract void ExtractKeys(ref SessionParseState state, int startIndex, List keys); } /// @@ -381,7 +382,7 @@ public FindKeysRange(int lastKey, int keyStep, int limit) : this() } /// - public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) + public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) { int lastKey; if (LastKey < 0) @@ -461,7 +462,7 @@ public FindKeysKeyNum(int keyNumIdx, int firstKey, int keyStep) : this() } /// - public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) + public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) { int numKeys = 0; int firstKey = startIndex + FirstKey; @@ -518,7 +519,7 @@ public override void ToRespFormat(ref RespMemoryWriter writer) } /// - public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) + public override void ExtractKeys(ref SessionParseState state, int startIndex, List keys) { // Do nothing } diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index c7fdbbc21ad..9c3901c5a90 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -19,18 +19,18 @@ namespace Garnet.server { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; - using LockableGarnetApi = GarnetApi, - SpanByteAllocator>>, - LockableContext>, - GenericAllocator>>>>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; + using TransactionalGarnetApi = GarnetApi, + SpanByteAllocator>>, + TransactionalContext, + ObjectAllocator>>>; /// /// RESP server session @@ -96,7 +96,7 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase /// public StorageSession storageSession; internal BasicGarnetApi basicGarnetApi; - internal LockableGarnetApi lockableGarnetApi; + internal TransactionalGarnetApi transactionalGarnetApi; internal TransactionManager txnManager; readonly IGarnetAuthenticator _authenticator; @@ -539,7 +539,7 @@ private void ProcessMessages() { if (txnManager.state == TxnState.Running) { - _ = ProcessBasicCommands(cmd, ref lockableGarnetApi); + _ = ProcessBasicCommands(cmd, ref transactionalGarnetApi); } else _ = cmd switch { @@ -1134,7 +1134,7 @@ ReadOnlySpan GetUpperCaseCommand(out bool success) return result; } - public ArgSlice GetCommandAsArgSlice(out bool success) + public PinnedSpanByte GetCommandAsArgSlice(out bool success) { if (bytesRead - readHead < 6) { @@ -1162,7 +1162,7 @@ public ArgSlice GetCommandAsArgSlice(out bool success) } Debug.Assert(*(recvBufferPtr + readHead + 1) == '\n'); - var result = new ArgSlice(recvBufferPtr + readHead + 2, psize); + var result = PinnedSpanByte.FromPinnedPointer(recvBufferPtr + readHead + 2, psize); Debug.Assert(*(recvBufferPtr + readHead + 2 + psize) == '\r'); Debug.Assert(*(recvBufferPtr + readHead + 2 + psize + 1) == '\n'); @@ -1191,13 +1191,11 @@ private unsafe bool Write(ref Status s, ref byte* dst, int length) private static unsafe bool Write(ref SpanByteAndMemory k, ref byte* dst, int length) { - if (k.Length > length) return false; + if (k.Length > length) + return false; - var dest = new SpanByte(length, (IntPtr)dst); - if (k.IsSpanByte) - k.SpanByte.CopyTo(ref dest); - else - k.AsMemoryReadOnlySpan().CopyTo(dest.AsSpan()); + k. + ReadOnlySpan.CopyTo(new Span(dst, length)); return true; } @@ -1597,7 +1595,7 @@ private GarnetDatabaseSession CreateDatabaseSession(int dbId) { var dbStorageSession = new StorageSession(storeWrapper, scratchBufferManager, sessionMetrics, LatencyMetrics, logger, dbId, respProtocolVersion); var dbGarnetApi = new BasicGarnetApi(dbStorageSession, dbStorageSession.basicContext, dbStorageSession.objectStoreBasicContext); - var dbLockableGarnetApi = new LockableGarnetApi(dbStorageSession, dbStorageSession.lockableContext, dbStorageSession.objectStoreLockableContext); + var dbLockableGarnetApi = new TransactionalGarnetApi(dbStorageSession, dbStorageSession.transactionalContext, dbStorageSession.objectStoreTransactionalContext); var transactionManager = new TransactionManager(storeWrapper, this, dbGarnetApi, dbLockableGarnetApi, dbStorageSession, scratchBufferManager, storeWrapper.serverOptions.EnableCluster, logger, dbId); @@ -1616,7 +1614,7 @@ private void SwitchActiveDatabaseSession(GarnetDatabaseSession dbSession) this.txnManager = dbSession.TransactionManager; this.storageSession = dbSession.StorageSession; this.basicGarnetApi = dbSession.GarnetApi; - this.lockableGarnetApi = dbSession.LockableGarnetApi; + this.transactionalGarnetApi = dbSession.TransactionalGarnetApi; this.storageSession.UpdateRespProtocolVersion(this.respProtocolVersion); } diff --git a/libs/server/Resp/RespServerSessionSlotVerify.cs b/libs/server/Resp/RespServerSessionSlotVerify.cs index 9de8ee1c18d..266a24e4963 100644 --- a/libs/server/Resp/RespServerSessionSlotVerify.cs +++ b/libs/server/Resp/RespServerSessionSlotVerify.cs @@ -3,6 +3,7 @@ using System; using System.Diagnostics; +using Tsavorite.core; namespace Garnet.server { @@ -18,7 +19,7 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase /// Whether caller is going to perform a readonly or read/write operation /// Key count if different than keys array length /// True when ownership is verified, false otherwise - bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, int count = -1) + bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, int count = -1) => clusterSession != null && clusterSession.NetworkKeyArraySlotVerify(keys, readOnly, SessionAsking, ref dcurr, ref dend, count); bool CanServeSlot(RespCommand cmd) diff --git a/libs/server/ArgSlice/ScratchBufferManager.cs b/libs/server/ScratchBufferManager.cs similarity index 86% rename from libs/server/ArgSlice/ScratchBufferManager.cs rename to libs/server/ScratchBufferManager.cs index 3e64a42a6f9..0113e009dbe 100644 --- a/libs/server/ArgSlice/ScratchBufferManager.cs +++ b/libs/server/ScratchBufferManager.cs @@ -8,6 +8,7 @@ using System.Runtime.InteropServices; using System.Text; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -53,7 +54,7 @@ public Span FullBuffer() /// Rewind (pop) the last entry of scratch buffer (rewinding the current scratch buffer offset), /// if it contains the given ArgSlice /// - public bool RewindScratchBuffer(ref ArgSlice slice) + public bool RewindScratchBuffer(PinnedSpanByte slice) { if (slice.ptr + slice.Length == scratchBufferHead + scratchBufferOffset) { @@ -81,11 +82,11 @@ public bool ResetScratchBuffer(int offset) /// /// Create ArgSlice in scratch buffer, from given ReadOnlySpan /// - public ArgSlice CreateArgSlice(ReadOnlySpan bytes) + public PinnedSpanByte CreateArgSlice(ReadOnlySpan bytes) { ExpandScratchBufferIfNeeded(bytes.Length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, bytes.Length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, bytes.Length); bytes.CopyTo(retVal.Span); scratchBufferOffset += bytes.Length; return retVal; @@ -103,12 +104,12 @@ public void MoveOffset(int length) /// /// Create ArgSlice in UTF8 format in scratch buffer, from given string /// - public ArgSlice CreateArgSlice(string str) + public PinnedSpanByte CreateArgSlice(string str) { int length = Encoding.UTF8.GetByteCount(str); ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); Encoding.UTF8.GetBytes(str, retVal.Span); scratchBufferOffset += length; return retVal; @@ -138,12 +139,12 @@ public ReadOnlySpan UTF8EncodeString(string str) /// /// Create an ArgSlice that includes a header of specified size, followed by RESP Bulk-String formatted versions of the specified ArgSlice values (arg1 and arg2) /// - public ArgSlice FormatScratchAsResp(int headerSize, ArgSlice arg1, ArgSlice arg2) + public PinnedSpanByte FormatScratchAsResp(int headerSize, PinnedSpanByte arg1, PinnedSpanByte arg2) { int length = headerSize + GetRespFormattedStringLength(arg1) + GetRespFormattedStringLength(arg2); ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); retVal.Span[..headerSize].Clear(); // Clear the header byte* ptr = scratchBufferHead + scratchBufferOffset + headerSize; @@ -160,12 +161,12 @@ public ArgSlice FormatScratchAsResp(int headerSize, ArgSlice arg1, ArgSlice arg2 /// /// Create an ArgSlice that includes a header of specified size, followed by RESP Bulk-String formatted versions of the specified ArgSlice value arg1 /// - public ArgSlice FormatScratchAsResp(int headerSize, ArgSlice arg1) + public PinnedSpanByte FormatScratchAsResp(int headerSize, PinnedSpanByte arg1) { int length = headerSize + GetRespFormattedStringLength(arg1); ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); retVal.Span[..headerSize].Clear(); // Clear the header byte* ptr = scratchBufferHead + scratchBufferOffset + headerSize; @@ -180,12 +181,12 @@ public ArgSlice FormatScratchAsResp(int headerSize, ArgSlice arg1) /// /// Create an ArgSlice that includes a header of specified size, followed by the specified ArgSlice (arg) /// - public ArgSlice FormatScratch(int headerSize, ArgSlice arg) + public PinnedSpanByte FormatScratch(int headerSize, PinnedSpanByte arg) { int length = headerSize + arg.Length; ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); retVal.Span[..headerSize].Clear(); // Clear the header byte* ptr = scratchBufferHead + scratchBufferOffset + headerSize; @@ -199,11 +200,11 @@ public ArgSlice FormatScratch(int headerSize, ArgSlice arg) /// /// Create an ArgSlice of specified length, leaves contents as is /// - public ArgSlice CreateArgSlice(int length) + public PinnedSpanByte CreateArgSlice(int length) { ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); scratchBufferOffset += length; Debug.Assert(scratchBufferOffset <= scratchBuffer.Length); return retVal; @@ -214,26 +215,26 @@ public ArgSlice CreateArgSlice(int length) /// Does NOT move the offset forward /// /// - public ArgSlice ViewRemainingArgSlice(int minLength = 0) + public PinnedSpanByte ViewRemainingArgSlice(int minLength = 0) { ExpandScratchBufferIfNeeded(minLength); - return new ArgSlice(scratchBufferHead + scratchBufferOffset, scratchBuffer.Length - scratchBufferOffset); + return PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, scratchBuffer.Length - scratchBufferOffset); } - public ArgSlice ViewFullArgSlice() + public PinnedSpanByte ViewFullArgSlice() { - return new ArgSlice(scratchBufferHead, scratchBufferOffset); + return PinnedSpanByte.FromPinnedPointer(scratchBufferHead, scratchBufferOffset); } /// /// Create an ArgSlice that includes a header of specified size, followed by the specified Memory /// - public ArgSlice FormatScratch(int headerSize, ReadOnlySpan arg) + public PinnedSpanByte FormatScratch(int headerSize, ReadOnlySpan arg) { int length = headerSize + arg.Length; ExpandScratchBufferIfNeeded(length); - var retVal = new ArgSlice(scratchBufferHead + scratchBufferOffset, length); + var retVal = PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset, length); retVal.Span[..headerSize].Clear(); // Clear the header byte* ptr = scratchBufferHead + scratchBufferOffset + headerSize; @@ -310,7 +311,7 @@ public void WriteArgument(ReadOnlySpan arg) /// /// /// - static int GetRespFormattedStringLength(ArgSlice slice) + static int GetRespFormattedStringLength(PinnedSpanByte slice) => 1 + NumUtils.CountDigits(slice.Length) + 2 + slice.Length + 2; void ExpandScratchBufferIfNeeded(int newLength) @@ -337,7 +338,7 @@ void ExpandScratchBuffer(int newLength, int? copyLengthOverride = null) } /// - /// Returns a new + /// Returns a new /// with the bytes of the buffer; /// these are the most recently added bytes. /// @@ -345,9 +346,9 @@ void ExpandScratchBuffer(int newLength, int? copyLengthOverride = null) /// This is called by functions that add multiple items to the buffer, /// after all items have been added and all reallocations have been done. /// - public ArgSlice GetSliceFromTail(int length) + public PinnedSpanByte GetSliceFromTail(int length) { - return new ArgSlice(scratchBufferHead + scratchBufferOffset - length, length); + return PinnedSpanByte.FromPinnedPointer(scratchBufferHead + scratchBufferOffset - length, length); } /// diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index cfd9232222f..00c88f278ff 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -289,7 +289,7 @@ public class GarnetServerOptions : ServerOptions public int CheckpointThrottleFlushDelayMs = 0; /// - /// Enable FastCommit mode for TsavoriteLog + /// Enable FastCommit mode for TsavoriteAof /// public bool EnableFastCommit = true; @@ -539,7 +539,7 @@ public void Initialize(ILoggerFactory loggerFactory = null) /// Tsavorite Log factory instance /// /// - public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, + public KVSettings GetSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, out INamedDeviceFactory logFactory) { if (MutablePercent is < 10 or > 95) @@ -547,7 +547,7 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, var indexCacheLines = IndexSizeCachelines("hash index size", IndexSize); - KVSettings kvSettings = new() + KVSettings kvSettings = new() { IndexSize = indexCacheLines * 64L, PreallocateLog = false, @@ -698,7 +698,7 @@ public static int MemorySizeBits(string memorySize, string storePageSize, out in /// /// Get KVSettings for the object store log /// - public KVSettings GetObjectStoreSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, + public KVSettings GetObjectStoreSettings(ILoggerFactory loggerFactory, LightEpoch epoch, StateMachineDriver stateMachineDriver, out long objHeapMemorySize, out long objReadCacheHeapMemorySize) { objReadCacheHeapMemorySize = default; @@ -707,7 +707,7 @@ public KVSettings GetObjectStoreSettings(ILoggerFactory l throw new Exception("ObjectStoreMutablePercent must be between 10 and 95"); var indexCacheLines = IndexSizeCachelines("object store hash index size", ObjectStoreIndexSize); - KVSettings kvSettings = new() + KVSettings kvSettings = new() { IndexSize = indexCacheLines * 64L, PreallocateLog = false, @@ -797,11 +797,10 @@ public KVSettings GetObjectStoreSettings(ILoggerFactory l } else if (UseRevivBinsPowerOf2 || RevivBinRecordSizes?.Length > 0) { - logger?.LogInformation("[Object Store] Using Revivification with a single fixed-size bin"); - kvSettings.RevivificationSettings = RevivificationSettings.DefaultFixedLength.Clone(); + logger?.LogInformation("[Store] Using Revivification with power-of-2 bins"); + kvSettings.RevivificationSettings = RevivificationSettings.PowerOf2Bins.Clone(); + kvSettings.RevivificationSettings.NumberOfBinsToSearch = RevivNumberOfBinsToSearch; kvSettings.RevivificationSettings.RevivifiableFraction = RevivifiableFraction; - kvSettings.RevivificationSettings.FreeRecordBins[0].NumberOfRecords = RevivObjBinRecordCount; - kvSettings.RevivificationSettings.FreeRecordBins[0].BestFitScanLimit = RevivBinBestFitScanLimit; } else { diff --git a/libs/server/Servers/IServerSerializer.cs b/libs/server/Servers/IServerSerializer.cs deleted file mode 100644 index 61e341f2efb..00000000000 --- a/libs/server/Servers/IServerSerializer.cs +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Garnet.server -{ - /// - /// Serializer interface for server-side processing - /// - /// Key - /// Value - /// Input - /// Output - public unsafe interface IServerSerializer - { - /// - /// Write element to given destination, with length bytes of space available - /// - /// Element to write - /// Destination memory - /// Space (bytes) available at destination - /// True if write succeeded, false if not (insufficient space) - bool Write(ref TKey k, ref byte* dst, int length); - - /// - /// Write element to given destination, with length bytes of space available - /// - /// Element to write - /// Destination memory - /// Space (bytes) available at destination - /// True if write succeeded, false if not (insufficient space) - bool Write(ref TValue v, ref byte* dst, int length); - - /// - /// Write element to given destination, with length bytes of space available - /// - /// Element to write - /// Destination memory - /// Space (bytes) available at destination - /// True if write succeeded, false if not (insufficient space) - bool Write(ref TOutput o, ref byte* dst, int length); - - /// - /// Get length of given output - /// - /// - /// - int GetLength(ref TOutput o); - - /// - /// Read key by reference, from given location - /// - /// Memory location - /// Key - ref TKey ReadKeyByRef(ref byte* src); - - /// - /// Read value by reference, from given location - /// - /// Memory location - /// Value - ref TValue ReadValueByRef(ref byte* src); - - /// - /// Read input by reference, from given location - /// - /// Memory location - /// Input - ref TInput ReadInputByRef(ref byte* src); - - /// - /// Read memory as output (by reference), at given location - /// - /// Memory location - /// Length of buffer at memory - /// Output - ref TOutput AsRefOutput(byte* src, int length); - - /// - /// Skip output (increment address) - /// - /// Memory location - void SkipOutput(ref byte* src); - } -} \ No newline at end of file diff --git a/libs/server/Servers/ServerOptions.cs b/libs/server/Servers/ServerOptions.cs index 953d47072ea..d827fafe32d 100644 --- a/libs/server/Servers/ServerOptions.cs +++ b/libs/server/Servers/ServerOptions.cs @@ -188,10 +188,10 @@ public int IndexSizeCachelines(string name, string indexSize) /// /// Get KVSettings /// - public void GetSettings() + public void GetSettings() { var indexCacheLines = IndexSizeCachelines("hash index size", IndexSize); - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = indexCacheLines * 64L, PreallocateLog = false, diff --git a/libs/server/SessionParseStateExtensions.cs b/libs/server/SessionParseStateExtensions.cs index b33bbda151a..7e505c3a813 100644 --- a/libs/server/SessionParseStateExtensions.cs +++ b/libs/server/SessionParseStateExtensions.cs @@ -5,6 +5,7 @@ using System.Collections.Generic; using System.Text; using Garnet.common; +using Tsavorite.core; namespace Garnet.server { @@ -154,25 +155,22 @@ internal static unsafe bool TryGetBitfieldEncoding(this SessionParseState parseS isSigned = default; var encodingSlice = parseState.GetArgSliceByRef(idx); - if (encodingSlice.length <= 1) + if (encodingSlice.Length <= 1) { return false; } - var ptr = encodingSlice.ptr + 1; - - isSigned = *encodingSlice.ptr == 'i'; - - if (!isSigned && *encodingSlice.ptr != 'u') - { + var ptr = encodingSlice.ToPointer() + 1; + byte b = *encodingSlice.ToPointer(); + isSigned = b == 'i'; + if (!isSigned && b != 'u') return false; - } return - RespReadUtils.TryReadInt64Safe(ref ptr, encodingSlice.ptr + encodingSlice.length, + RespReadUtils.TryReadInt64Safe(ref ptr, encodingSlice.ToPointer() + encodingSlice.Length, out bitCount, out var bytesRead, out _, out _, allowLeadingZeros: false) && - ((int)bytesRead == encodingSlice.length - 1) && (bytesRead > 0L) && + ((int)bytesRead == encodingSlice.Length - 1) && (bytesRead > 0L) && (bitCount > 0) && ((isSigned && bitCount <= 64) || (!isSigned && bitCount < 64)); @@ -197,12 +195,12 @@ internal static unsafe bool TryGetBitfieldOffset(this SessionParseState parseSta return false; } - var ptr = offsetSlice.ptr; - var len = offsetSlice.length; + var ptr = offsetSlice.ToPointer(); + var len = offsetSlice.Length; if (*ptr == '#') { - if (offsetSlice.length == 1) + if (offsetSlice.Length == 1) return false; multiplyOffset = true; @@ -211,7 +209,7 @@ internal static unsafe bool TryGetBitfieldOffset(this SessionParseState parseSta } return - RespReadUtils.TryReadInt64Safe(ref ptr, offsetSlice.ptr + offsetSlice.length, + RespReadUtils.TryReadInt64Safe(ref ptr, offsetSlice.ToPointer() + offsetSlice.Length, out bitFieldOffset, out var bytesRead, out _, out _, allowLeadingZeros: false) && ((int)bytesRead == len) && (bytesRead > 0L) && @@ -251,7 +249,7 @@ public static bool TryGetGeoSearchOptions(this SessionParseState parseState, if (command == RespCommand.GEORADIUSBYMEMBER || command == RespCommand.GEORADIUSBYMEMBER_RO) { // From Member - searchOpts.fromMember = parseState.GetArgSliceByRef(currTokenIdx++).SpanByte.ToByteArray(); + searchOpts.fromMember = parseState.GetArgSliceByRef(currTokenIdx++).ToArray(); searchOpts.origin = GeoOriginType.FromMember; } else @@ -308,7 +306,7 @@ public static bool TryGetGeoSearchOptions(this SessionParseState parseState, break; } - searchOpts.fromMember = parseState.GetArgSliceByRef(currTokenIdx++).SpanByte.ToByteArray(); + searchOpts.fromMember = parseState.GetArgSliceByRef(currTokenIdx++).ToArray(); searchOpts.origin = GeoOriginType.FromMember; continue; } @@ -761,7 +759,7 @@ internal static bool TryGetGeoLonLat(this SessionParseState parseState, int idx, /// 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) + internal static bool TryExtractKeysFromSpecs(this ref SessionParseState state, RespCommandKeySpecification[] keySpecs, out List keys) { keys = new(); @@ -784,7 +782,7 @@ internal static bool TryExtractKeysFromSpecs(this ref SessionParseState state, R /// 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) + internal static bool TryExtractKeysAndFlagsFromSpecs(this ref SessionParseState state, RespCommandKeySpecification[] keySpecs, out List keys, out List flags) { keys = new(); flags = new(); @@ -814,7 +812,7 @@ internal static bool TryExtractKeysAndFlagsFromSpecs(this ref SessionParseState /// 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) + private static bool ExtractKeysFromSpec(ref SessionParseState state, List keys, RespCommandKeySpecification spec) { int startIndex = 0; diff --git a/libs/server/Sessions/ServerSessionBase.cs b/libs/server/Sessions/ServerSessionBase.cs index 9c6cc852718..93eeeecdb1d 100644 --- a/libs/server/Sessions/ServerSessionBase.cs +++ b/libs/server/Sessions/ServerSessionBase.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using Garnet.networking; +using Tsavorite.core; namespace Garnet.server { @@ -38,7 +39,7 @@ public ServerSessionBase(INetworkSender networkSender) /// /// /// - public abstract unsafe void Publish(ArgSlice key, ArgSlice value); + public abstract unsafe void Publish(PinnedSpanByte key, PinnedSpanByte value); /// /// Publish an update to a key to all the (pattern) subscribers of the key @@ -46,7 +47,7 @@ public ServerSessionBase(INetworkSender networkSender) /// /// /// - public abstract unsafe void PatternPublish(ArgSlice pattern, ArgSlice key, ArgSlice value); + public abstract unsafe void PatternPublish(PinnedSpanByte pattern, PinnedSpanByte key, PinnedSpanByte value); /// /// Dispose diff --git a/libs/server/SpanByteFunctionsForServer.cs b/libs/server/SpanByteFunctionsForServer.cs deleted file mode 100644 index 917eccb4031..00000000000 --- a/libs/server/SpanByteFunctionsForServer.cs +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Buffers; -using Tsavorite.core; - -namespace Garnet.server -{ - /// - /// Callback functions using SpanByteAndMemory output, for SpanByte key, value, input - /// - public class SpanByteFunctionsForServer : SpanByteFunctions - { - /// - /// Memory pool - /// - protected readonly MemoryPool memoryPool; - - /// - /// Constructor - /// - /// - public SpanByteFunctionsForServer(MemoryPool memoryPool = default) - { - this.memoryPool = memoryPool ?? MemoryPool.Shared; - } - - /// - public override bool SingleReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) - => CopyWithHeaderTo(ref value, ref dst, memoryPool); - - /// - public override bool ConcurrentReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - => CopyWithHeaderTo(ref value, ref dst, memoryPool); - - /// - /// Copy to given SpanByteAndMemory (header length and payload copied to actual span/memory) - /// - /// - /// - /// - private static unsafe bool CopyWithHeaderTo(ref SpanByte src, ref SpanByteAndMemory dst, MemoryPool memoryPool) - { - if (dst.IsSpanByte) - { - if (dst.Length >= src.TotalSize) - { - dst.Length = src.TotalSize; - var span = dst.SpanByte.AsSpan(); - fixed (byte* ptr = span) - *(int*)ptr = src.Length; - src.AsReadOnlySpan().CopyTo(span.Slice(sizeof(int))); - return true; - } - dst.ConvertToHeap(); - } - - dst.Length = src.TotalSize; - dst.Memory = memoryPool.Rent(src.TotalSize); - dst.Length = src.TotalSize; - fixed (byte* ptr = dst.Memory.Memory.Span) - *(int*)ptr = src.Length; - src.AsReadOnlySpan().CopyTo(dst.Memory.Memory.Span.Slice(sizeof(int))); - return true; - } - } -} \ No newline at end of file diff --git a/libs/server/SpanByteServerSerializer.cs b/libs/server/SpanByteServerSerializer.cs deleted file mode 100644 index b40d1ece97f..00000000000 --- a/libs/server/SpanByteServerSerializer.cs +++ /dev/null @@ -1,99 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Runtime.CompilerServices; -using Tsavorite.core; - -namespace Garnet.server -{ - /// - /// Serializer for SpanByte. Used only on server-side. - /// - public sealed unsafe class SpanByteServerSerializer : IServerSerializer - { - readonly int keyLength; - readonly int valueLength; - - [ThreadStatic] - static SpanByteAndMemory output; - - /// - /// Constructor - /// - /// Max key length - /// Max value length - public SpanByteServerSerializer(int maxKeyLength = 512, int maxValueLength = 512) - { - keyLength = maxKeyLength; - valueLength = maxValueLength; - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref SpanByte ReadKeyByRef(ref byte* src) - { - ref var ret = ref Unsafe.AsRef(src); - src += ret.TotalSize; - return ref ret; - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref SpanByte ReadValueByRef(ref byte* src) - { - ref var ret = ref Unsafe.AsRef(src); - src += ret.TotalSize; - return ref ret; - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref SpanByte ReadInputByRef(ref byte* src) - { - ref var ret = ref Unsafe.AsRef(src); - src += ret.TotalSize; - return ref ret; - } - - /// - public bool Write(ref SpanByte k, ref byte* dst, int length) - { - if (k.Length > length) return false; - - *(int*)dst = k.Length; - dst += sizeof(int); - var dest = new SpanByte(k.Length, (IntPtr)dst); - k.CopyTo(ref dest); - dst += k.Length; - return true; - } - - - /// - public bool Write(ref SpanByteAndMemory k, ref byte* dst, int length) - { - if (k.Length > length) return false; - - var dest = new SpanByte(length, (IntPtr)dst); - if (k.IsSpanByte) - k.SpanByte.CopyTo(ref dest); - else - k.AsMemoryReadOnlySpan().CopyTo(dest.AsSpan()); - return true; - } - - /// - public ref SpanByteAndMemory AsRefOutput(byte* src, int length) - { - output = SpanByteAndMemory.FromPinnedSpan(new Span(src, length)); - return ref output; - } - - /// - public void SkipOutput(ref byte* src) => src += (*(int*)src) + sizeof(int); - - /// - public int GetLength(ref SpanByteAndMemory o) => o.Length; - } -} \ No newline at end of file diff --git a/libs/server/Storage/Functions/EtagState.cs b/libs/server/Storage/Functions/EtagState.cs index 244ff325ffa..90078009c49 100644 --- a/libs/server/Storage/Functions/EtagState.cs +++ b/libs/server/Storage/Functions/EtagState.cs @@ -5,59 +5,26 @@ namespace Garnet.server { - internal static class EtagConstants - { - public const byte EtagSize = sizeof(long); - - public const long NoETag = 0; - } - /// /// Indirection wrapper to provide a way to set offsets related to Etags and use the getters opaquely from outside. /// - public struct EtagState + public struct ETagState { - public EtagState() - { - } - - /// - /// Offset used accounting space for an etag during allocation - /// - public byte etagOffsetForVarlen { get; set; } = 0; - - /// - /// Gives an offset used to opaquely work with Etag in a payload. By calling this you can skip past the etag if it is present. - /// - public byte etagSkippedStart { get; private set; } = 0; - - /// - /// Resp response methods depend on the value for end being -1 or length of the payload. This field lets you work with providing the end opaquely. - /// - public int etagAccountedLength { get; private set; } = -1; + public ETagState() { } /// /// Field provides access to getting an Etag from a record, hiding whether it is actually present or not. /// - public long etag { get; set; } = EtagConstants.NoETag; + public long ETag { get; set; } = LogRecord.NoETag; /// /// Sets the values to indicate the presence of an Etag as a part of the payload value /// - public static void SetValsForRecordWithEtag(ref EtagState curr, ref SpanByte value) - { - curr.etagOffsetForVarlen = EtagConstants.EtagSize; - curr.etagSkippedStart = EtagConstants.EtagSize; - curr.etagAccountedLength = value.LengthWithoutMetadata; - curr.etag = value.GetEtagInPayload(); - } + public static void SetValsForRecordWithEtag(ref ETagState curr, ref TSourceLogRecord srcLogRecord) + where TSourceLogRecord : ISourceLogRecord + => curr.ETag = srcLogRecord.ETag; - public static void ResetState(ref EtagState curr) - { - curr.etagOffsetForVarlen = 0; - curr.etagSkippedStart = 0; - curr.etag = EtagConstants.NoETag; - curr.etagAccountedLength = -1; - } + public static void ResetState(ref ETagState curr) + => curr.ETag = LogRecord.NoETag; } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/FunctionsState.cs b/libs/server/Storage/Functions/FunctionsState.cs index 4ef24a38260..38891071819 100644 --- a/libs/server/Storage/Functions/FunctionsState.cs +++ b/libs/server/Storage/Functions/FunctionsState.cs @@ -3,6 +3,8 @@ using System; using System.Buffers; +using Garnet.common; +using Microsoft.Extensions.Logging; using Tsavorite.core; namespace Garnet.server @@ -19,14 +21,15 @@ internal sealed class FunctionsState public readonly MemoryPool memoryPool; public readonly CacheSizeTracker objectStoreSizeTracker; public readonly GarnetObjectSerializer garnetObjectSerializer; - public EtagState etagState; + public ETagState etagState; + public readonly ILogger logger; public byte respProtocolVersion; public bool StoredProcMode; internal ReadOnlySpan nilResp => respProtocolVersion >= 3 ? CmdStrings.RESP3_NULL_REPLY : CmdStrings.RESP_ERRNOTFOUND; public FunctionsState(TsavoriteLog appendOnlyFile, WatchVersionMap watchVersionMap, CustomCommandManager customCommandManager, - MemoryPool memoryPool, CacheSizeTracker objectStoreSizeTracker, GarnetObjectSerializer garnetObjectSerializer, + MemoryPool memoryPool, CacheSizeTracker objectStoreSizeTracker, GarnetObjectSerializer garnetObjectSerializer, ILogger logger, byte respProtocolVersion = ServerOptions.DEFAULT_RESP_VERSION) { this.appendOnlyFile = appendOnlyFile; @@ -35,10 +38,50 @@ public FunctionsState(TsavoriteLog appendOnlyFile, WatchVersionMap watchVersionM this.memoryPool = memoryPool ?? MemoryPool.Shared; this.objectStoreSizeTracker = objectStoreSizeTracker; this.garnetObjectSerializer = garnetObjectSerializer; - this.etagState = new EtagState(); + this.etagState = new ETagState(); + this.logger = logger; this.respProtocolVersion = respProtocolVersion; } + internal void CopyDefaultResp(ReadOnlySpan resp, ref SpanByteAndMemory dst) + { + if (resp.Length < dst.SpanByte.Length) + { + resp.CopyTo(dst.SpanByte.Span); + dst.SpanByte.Length = resp.Length; + return; + } + + dst.ConvertToHeap(); + dst.Length = resp.Length; + dst.Memory = memoryPool.Rent(resp.Length); + resp.CopyTo(dst.MemorySpan); + } + + internal unsafe void CopyRespNumber(long number, ref SpanByteAndMemory dst) + { + byte* curr = dst.SpanByte.ToPointer(); + byte* end = curr + dst.SpanByte.Length; + if (RespWriteUtils.TryWriteInt64(number, ref curr, end, out int integerLen, out int totalLen)) + { + dst.SpanByte.Length = (int)(curr - dst.SpanByte.ToPointer()); + return; + } + + //handle resp buffer overflow here + dst.ConvertToHeap(); + dst.Length = totalLen; + dst.Memory = memoryPool.Rent(totalLen); + fixed (byte* ptr = dst.MemorySpan) + { + byte* cc = ptr; + *cc++ = (byte)':'; + NumUtils.WriteInt64(number, integerLen, ref cc); + *cc++ = (byte)'\r'; + *cc++ = (byte)'\n'; + } + } + public CustomRawStringFunctions GetCustomCommandFunctions(int id) => customCommandManager.TryGetCustomCommand(id, out var cmd) ? cmd.functions : null; diff --git a/libs/server/Storage/Functions/MainStore/CallbackMethods.cs b/libs/server/Storage/Functions/MainStore/CallbackMethods.cs index 4882e2e5cbb..63665d3f13c 100644 --- a/libs/server/Storage/Functions/MainStore/CallbackMethods.cs +++ b/libs/server/Storage/Functions/MainStore/CallbackMethods.cs @@ -8,15 +8,15 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// - public void ReadCompletionCallback(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, Status status, RecordMetadata recordMetadata) + public void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, Status status, RecordMetadata recordMetadata) { } /// - public void RMWCompletionCallback(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, Status status, RecordMetadata recordMetadata) + public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, Status status, RecordMetadata recordMetadata) { } } diff --git a/libs/server/Storage/Functions/MainStore/DeleteMethods.cs b/libs/server/Storage/Functions/MainStore/DeleteMethods.cs index 6c055bd3682..254f529a306 100644 --- a/libs/server/Storage/Functions/MainStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/MainStore/DeleteMethods.cs @@ -8,31 +8,31 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// - public bool SingleDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - recordInfo.ClearHasETag(); + logRecord.InfoRef.ClearHasETag(); functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); return true; } /// - public void PostSingleDeleter(ref SpanByte key, ref DeleteInfo deleteInfo) + public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { if (functionsState.appendOnlyFile != null) - WriteLogDelete(ref key, deleteInfo.Version, deleteInfo.SessionID); + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); } /// - public bool ConcurrentDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - recordInfo.ClearHasETag(); - if (!deleteInfo.RecordInfo.Modified) + logRecord.ClearOptionals(); + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogDelete(ref key, deleteInfo.Version, deleteInfo.SessionID); + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); return true; } } diff --git a/libs/server/Storage/Functions/MainStore/MainSessionFunctions.cs b/libs/server/Storage/Functions/MainStore/MainSessionFunctions.cs index 656d71e1e81..fe4ecae7873 100644 --- a/libs/server/Storage/Functions/MainStore/MainSessionFunctions.cs +++ b/libs/server/Storage/Functions/MainStore/MainSessionFunctions.cs @@ -8,7 +8,7 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { readonly FunctionsState functionsState; diff --git a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs index 57a01112e2d..237427eaa43 100644 --- a/libs/server/Storage/Functions/MainStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/MainStore/PrivateMethods.cs @@ -5,6 +5,7 @@ using System.Buffers; using System.Diagnostics; using Garnet.common; +using Microsoft.Extensions.Logging; using Tsavorite.core; namespace Garnet.server @@ -12,18 +13,18 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { - static void CopyTo(ref SpanByte src, ref SpanByteAndMemory dst, MemoryPool memoryPool) + static void CopyTo(ReadOnlySpan src, ref SpanByteAndMemory dst, MemoryPool memoryPool) { - int srcLength = src.LengthWithoutMetadata; + int srcLength = src.Length; if (dst.IsSpanByte) { if (dst.Length >= srcLength) { dst.Length = srcLength; - src.AsReadOnlySpan().CopyTo(dst.SpanByte.AsSpan()); + src.CopyTo(dst.SpanByte.Span); return; } dst.ConvertToHeap(); @@ -31,20 +32,20 @@ static void CopyTo(ref SpanByte src, ref SpanByteAndMemory dst, MemoryPool dst.Memory = memoryPool.Rent(srcLength); dst.Length = srcLength; - src.AsReadOnlySpan().CopyTo(dst.Memory.Memory.Span); + src.CopyTo(dst.MemorySpan); } - void CopyRespTo(ref SpanByte src, ref SpanByteAndMemory dst, int start = 0, int end = -1) + void CopyRespTo(ReadOnlySpan src, ref SpanByteAndMemory dst, int start = 0, int end = -1) { - int srcLength = end == -1 ? src.LengthWithoutMetadata : ((start < end) ? (end - start) : 0); + int srcLength = end == -1 ? src.Length : ((start < end) ? (end - start) : 0); if (srcLength == 0) { - CopyDefaultResp(CmdStrings.RESP_EMPTY, ref dst); + functionsState.CopyDefaultResp(CmdStrings.RESP_EMPTY, ref dst); return; } var numLength = NumUtils.CountDigits(srcLength); - int totalSize = 1 + numLength + 2 + srcLength + 2; // $5\r\nvalue\r\n + var totalSize = 1 + numLength + 2 + srcLength + 2; // $5\r\nvalue\r\n if (dst.IsSpanByte) { @@ -52,12 +53,12 @@ void CopyRespTo(ref SpanByte src, ref SpanByteAndMemory dst, int start = 0, int { dst.Length = totalSize; - byte* tmp = dst.SpanByte.ToPointer(); + var tmp = dst.SpanByte.ToPointer(); *tmp++ = (byte)'$'; NumUtils.WriteInt32(srcLength, numLength, ref tmp); *tmp++ = (byte)'\r'; *tmp++ = (byte)'\n'; - src.AsReadOnlySpan().Slice(start, srcLength).CopyTo(new Span(tmp, srcLength)); + src.Slice(start, srcLength).CopyTo(new Span(tmp, srcLength)); tmp += srcLength; *tmp++ = (byte)'\r'; *tmp++ = (byte)'\n'; @@ -68,22 +69,25 @@ void CopyRespTo(ref SpanByte src, ref SpanByteAndMemory dst, int start = 0, int dst.Memory = functionsState.memoryPool.Rent(totalSize); dst.Length = totalSize; - fixed (byte* ptr = dst.Memory.Memory.Span) + fixed (byte* ptr = dst.MemorySpan) { - byte* tmp = ptr; + var tmp = ptr; *tmp++ = (byte)'$'; NumUtils.WriteInt32(srcLength, numLength, ref tmp); *tmp++ = (byte)'\r'; *tmp++ = (byte)'\n'; - src.AsReadOnlySpan().Slice(start, srcLength).CopyTo(new Span(tmp, srcLength)); + src.Slice(start, srcLength).CopyTo(new Span(tmp, srcLength)); tmp += srcLength; *tmp++ = (byte)'\r'; *tmp++ = (byte)'\n'; } } - void CopyRespToWithInput(ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory dst, bool isFromPending) + void CopyRespToWithInput(ref TSourceLogRecord srcLogRecord, ref RawStringInput input, ref SpanByteAndMemory output, bool isFromPending) + where TSourceLogRecord : ISourceLogRecord { + var value = srcLogRecord.ValueSpan; + switch (input.header.cmd) { case RespCommand.ASYNC: @@ -91,55 +95,43 @@ void CopyRespToWithInput(ref RawStringInput input, ref SpanByte value, ref SpanB // to the network buffer in case the operation does go pending (latter is indicated by isFromPending) // This is accomplished by calling ConvertToHeap on the destination SpanByteAndMemory if (isFromPending) - dst.ConvertToHeap(); - CopyRespTo(ref value, ref dst, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + output.ConvertToHeap(); + CopyRespTo(value, ref output); break; case RespCommand.MIGRATE: - if (value.Length <= dst.Length) - { - value.CopyTo(ref dst.SpanByte); - dst.Length = value.Length; - return; - } - - dst.ConvertToHeap(); - dst.Length = value.TotalSize; - - if (dst.Memory == default) // Allocate new heap buffer - dst.Memory = functionsState.memoryPool.Rent(dst.Length); - else if (dst.Memory.Memory.Span.Length < value.TotalSize) - // Allocate new heap buffer only if existing one is smaller - // otherwise it is safe to re-use existing buffer - { - dst.Memory.Dispose(); - dst.Memory = functionsState.memoryPool.Rent(dst.Length); - } - value.CopyTo(dst.Memory.Memory.Span); + DiskLogRecord.Serialize(ref srcLogRecord, valueSerializer: null, ref output, functionsState.memoryPool); break; case RespCommand.GET: // Get value without RESP header; exclude expiration - if (value.LengthWithoutMetadata <= dst.Length) + if (value.Length <= output.Length) { - dst.Length = value.LengthWithoutMetadata - functionsState.etagState.etagSkippedStart; - value.AsReadOnlySpan(functionsState.etagState.etagSkippedStart).CopyTo(dst.SpanByte.AsSpan()); + output.Length = value.Length; + value.CopyTo(output.SpanByte.Span); return; } - dst.ConvertToHeap(); - dst.Length = value.LengthWithoutMetadata - functionsState.etagState.etagSkippedStart; - dst.Memory = functionsState.memoryPool.Rent(value.LengthWithoutMetadata); - value.AsReadOnlySpan(functionsState.etagState.etagSkippedStart).CopyTo(dst.Memory.Memory.Span); + output.ConvertToHeap(); + output.Length = value.Length; + output.Memory = functionsState.memoryPool.Rent(value.Length); + value.CopyTo(output.MemorySpan); break; case RespCommand.GETBIT: var offset = input.arg1; - var oldValSet = BitmapManager.GetBit(offset, value.ToPointer() + functionsState.etagState.etagSkippedStart, value.Length - functionsState.etagState.etagSkippedStart); + byte oldValSet; + + if (srcLogRecord.IsPinnedValue) + oldValSet = BitmapManager.GetBit(offset, srcLogRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + oldValSet = BitmapManager.GetBit(offset, valuePtr, value.Length); + if (oldValSet == 0) - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref dst); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); else - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref dst); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output); break; case RespCommand.BITCOUNT: @@ -159,8 +151,14 @@ void CopyRespToWithInput(ref RawStringInput input, ref SpanByte value, ref SpanB } } - var count = BitmapManager.BitCountDriver(bcStartOffset, bcEndOffset, bcOffsetType, value.ToPointer() + functionsState.etagState.etagSkippedStart, value.Length - functionsState.etagState.etagSkippedStart); - CopyRespNumber(count, ref dst); + long count; + if (srcLogRecord.IsPinnedValue) + count = BitmapManager.BitCountDriver(bcStartOffset, bcEndOffset, bcOffsetType, srcLogRecord.PinnedValuePointer, value.Length); + else + fixed(byte* valuePtr = value) + count = BitmapManager.BitCountDriver(bcStartOffset, bcEndOffset, bcOffsetType, valuePtr, value.Length); + + functionsState.CopyRespNumber(count, ref output); break; case RespCommand.BITPOS: @@ -184,89 +182,118 @@ void CopyRespToWithInput(ref RawStringInput input, ref SpanByte value, ref SpanB } } - var pos = BitmapManager.BitPosDriver( - input: value.ToPointer() + functionsState.etagState.etagSkippedStart, - inputLen: value.Length - functionsState.etagState.etagSkippedStart, - startOffset: bpStartOffset, - endOffset: bpEndOffset, - searchFor: bpSetVal, - offsetType: bpOffsetType - ); - *(long*)dst.SpanByte.ToPointer() = pos; - CopyRespNumber(pos, ref dst); + long pos; + if (srcLogRecord.IsPinnedValue) + pos = BitmapManager.BitPosDriver(input: srcLogRecord.PinnedValuePointer, inputLen: value.Length, startOffset: bpStartOffset, + endOffset: bpEndOffset, searchFor: bpSetVal, offsetType: bpOffsetType); + else + fixed (byte* valuePtr = value) + pos = BitmapManager.BitPosDriver(input: valuePtr, inputLen: value.Length, startOffset: bpStartOffset, + endOffset: bpEndOffset, searchFor: bpSetVal, offsetType: bpOffsetType); + + *(long*)output.SpanByte.ToPointer() = pos; + functionsState.CopyRespNumber(pos, ref output); break; case RespCommand.BITOP: - var bitmap = (IntPtr)value.ToPointer() + functionsState.etagState.etagSkippedStart; - var output = dst.SpanByte.ToPointer(); + var outPtr = output.SpanByte.ToPointer(); - *(long*)output = bitmap.ToInt64(); - *(int*)(output + 8) = value.Length; + if (srcLogRecord.IsPinnedValue) + *(long*)outPtr = ((IntPtr)srcLogRecord.PinnedValuePointer).ToInt64(); + else + fixed(byte* valuePtr = value) + *(long*)outPtr = ((IntPtr)valuePtr).ToInt64(); + *(int*)(outPtr + sizeof(long)) = value.Length; return; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - var (retValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, - value.ToPointer() + functionsState.etagState.etagSkippedStart, - value.Length - functionsState.etagState.etagSkippedStart); + + long retValue; + bool overflow; + if (srcLogRecord.IsPinnedValue) + (retValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, srcLogRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + (retValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, valuePtr, value.Length); + if (!overflow) - CopyRespNumber(retValue, ref dst); + functionsState.CopyRespNumber(retValue, ref output); else - CopyDefaultResp(functionsState.nilResp, ref dst); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); return; case RespCommand.BITFIELD_RO: var bitFieldArgs_RO = GetBitFieldArguments(ref input); - var retValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, - value.ToPointer() + functionsState.etagState.etagSkippedStart, - value.Length - functionsState.etagState.etagSkippedStart); - CopyRespNumber(retValue_RO, ref dst); + + long retValue_RO; + if (srcLogRecord.IsPinnedValue) + retValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, srcLogRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + retValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, valuePtr, value.Length); + + functionsState.CopyRespNumber(retValue_RO, ref output); return; case RespCommand.PFCOUNT: case RespCommand.PFMERGE: - if (!HyperLogLog.DefaultHLL.IsValidHYLL(value.ToPointer(), value.Length)) + bool ok; + if (srcLogRecord.IsPinnedValue) + { + if (ok = HyperLogLog.DefaultHLL.IsValidHYLL(srcLogRecord.PinnedValuePointer, value.Length)) + Buffer.MemoryCopy(srcLogRecord.PinnedValuePointer, output.SpanByte.ToPointer(), value.Length, value.Length); + } + else + { + fixed (byte* valuePtr = value) + { + if (ok = HyperLogLog.DefaultHLL.IsValidHYLL(valuePtr, value.Length)) + Buffer.MemoryCopy(valuePtr, output.SpanByte.ToPointer(), value.Length, value.Length); + } + } + + if (!ok) { - *(long*)dst.SpanByte.ToPointer() = -1; + *(long*)output.SpanByte.ToPointer() = -1; return; } - if (value.Length <= dst.Length) + if (value.Length <= output.Length) { - Buffer.MemoryCopy(value.ToPointer(), dst.SpanByte.ToPointer(), value.Length, value.Length); - dst.SpanByte.Length = value.Length; + output.SpanByte.Length = value.Length; return; } throw new GarnetException($"Not enough space in {input.header.cmd} buffer"); case RespCommand.TTL: - var ttlValue = ConvertUtils.SecondsFromDiffUtcNowTicks(value.MetadataSize > 0 ? value.ExtraMetadata : -1); - CopyRespNumber(ttlValue, ref dst); + var ttlValue = ConvertUtils.SecondsFromDiffUtcNowTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(ttlValue, ref output); return; case RespCommand.PTTL: - var pttlValue = ConvertUtils.MillisecondsFromDiffUtcNowTicks(value.MetadataSize > 0 ? value.ExtraMetadata : -1); - CopyRespNumber(pttlValue, ref dst); + var pttlValue = ConvertUtils.MillisecondsFromDiffUtcNowTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(pttlValue, ref output); return; case RespCommand.GETRANGE: - var len = value.LengthWithoutMetadata - functionsState.etagState.etagSkippedStart; + var len = value.Length; var start = input.parseState.GetInt(0); var end = input.parseState.GetInt(1); (start, end) = NormalizeRange(start, end, len); - CopyRespTo(ref value, ref dst, start + functionsState.etagState.etagSkippedStart, end + functionsState.etagState.etagSkippedStart); + CopyRespTo(value, ref output, start, end); return; case RespCommand.EXPIRETIME: - var expireTime = ConvertUtils.UnixTimeInSecondsFromTicks(value.MetadataSize > 0 ? value.ExtraMetadata : -1); - CopyRespNumber(expireTime, ref dst); + var expireTime = ConvertUtils.UnixTimeInSecondsFromTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(expireTime, ref output); return; case RespCommand.PEXPIRETIME: - var pexpireTime = ConvertUtils.UnixTimeInMillisecondsFromTicks(value.MetadataSize > 0 ? value.ExtraMetadata : -1); - CopyRespNumber(pexpireTime, ref dst); + var pexpireTime = ConvertUtils.UnixTimeInMillisecondsFromTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(pexpireTime, ref output); return; default: @@ -274,43 +301,40 @@ void CopyRespToWithInput(ref RawStringInput input, ref SpanByte value, ref SpanB } } - bool EvaluateExpireInPlace(ExpireOption optionType, bool expiryExists, long newExpiry, ref SpanByte value, ref SpanByteAndMemory output) + bool EvaluateExpireInPlace(ref LogRecord logRecord, ExpireOption optionType, long newExpiry, ref SpanByteAndMemory output) { - ObjectOutputHeader* o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); - if (expiryExists) + var o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); + o->result1 = 0; + if (logRecord.Info.HasExpiration) { switch (optionType) { case ExpireOption.NX: - o->result1 = 0; - break; + return true; case ExpireOption.XX: case ExpireOption.None: - value.ExtraMetadata = newExpiry; + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + return true; case ExpireOption.GT: case ExpireOption.XXGT: - var replace = newExpiry < value.ExtraMetadata; - value.ExtraMetadata = replace ? value.ExtraMetadata : newExpiry; - if (replace) - o->result1 = 0; - else + if (newExpiry > logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; case ExpireOption.LT: case ExpireOption.XXLT: - replace = newExpiry > value.ExtraMetadata; - value.ExtraMetadata = replace ? value.ExtraMetadata : newExpiry; - if (replace) - o->result1 = 0; - else + if (newExpiry < logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; default: - throw new GarnetException($"EvaluateExpireInPlace exception expiryExists:{expiryExists}, optionType{optionType}"); + throw new GarnetException($"EvaluateExpireInPlace exception expiryExists: True, optionType {optionType}"); } - return true; } else { @@ -318,76 +342,87 @@ bool EvaluateExpireInPlace(ExpireOption optionType, bool expiryExists, long newE { case ExpireOption.NX: case ExpireOption.None: - case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite - return false; + case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite, so the new value must be less + var ok = logRecord.TrySetExpiration(newExpiry); + o->result1 = 1; + return ok; case ExpireOption.XX: - case ExpireOption.GT: + 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: - o->result1 = 0; return true; default: - throw new GarnetException($"EvaluateExpireInPlace exception expiryExists:{expiryExists}, optionType{optionType}"); + throw new GarnetException($"EvaluateExpireInPlace exception expiryExists: False, optionType {optionType}"); } } } - void EvaluateExpireCopyUpdate(ExpireOption optionType, bool expiryExists, long newExpiry, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output) + bool EvaluateExpireCopyUpdate(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ExpireOption optionType, long newExpiry, ReadOnlySpan newValue, ref SpanByteAndMemory output) { - ObjectOutputHeader* o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); + var expiryExists = logRecord.Info.HasExpiration; + var o = (ObjectOutputHeader*)output.SpanByte.ToPointer(); + o->result1 = 0; + + // TODO ETag? + if (!logRecord.TrySetValueSpan(newValue, ref sizeInfo)) + { + functionsState.logger?.LogError("Failed to set value in {methodName}", "EvaluateExpireCopyUpdate"); + return false; + } if (expiryExists) { + // Expiration already exists so there is no need to check for space (i.e. failure of TrySetExpiration) switch (optionType) { case ExpireOption.NX: - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - break; + return true; case ExpireOption.XX: case ExpireOption.None: - newValue.ExtraMetadata = newExpiry; - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + return true; case ExpireOption.GT: case ExpireOption.XXGT: - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - bool replace = newExpiry < oldValue.ExtraMetadata; - newValue.ExtraMetadata = replace ? oldValue.ExtraMetadata : newExpiry; - if (replace) - o->result1 = 0; - else + if (newExpiry > logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; case ExpireOption.LT: case ExpireOption.XXLT: - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - replace = newExpiry > oldValue.ExtraMetadata; - newValue.ExtraMetadata = replace ? oldValue.ExtraMetadata : newExpiry; - if (replace) - o->result1 = 0; - else + if (newExpiry < logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; + default: + throw new GarnetException($"EvaluateExpireCopyUpdate exception expiryExists:{expiryExists}, 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 - newValue.ExtraMetadata = newExpiry; - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); + if (!logRecord.TrySetExpiration(newExpiry)) + { + functionsState.logger?.LogError("Failed to add expiration in {methodName}.{caseName}", "EvaluateExpireCopyUpdate", "LT"); + return false; + } o->result1 = 1; - break; + return true; case ExpireOption.XX: case ExpireOption.GT: case ExpireOption.XXGT: case ExpireOption.XXLT: - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - o->result1 = 0; - break; + return true; + default: + throw new GarnetException($"EvaluateExpireCopyUpdate exception expiryExists:{expiryExists}, optionType{optionType}"); } } } @@ -415,52 +450,62 @@ void EvaluateExpireCopyUpdate(ExpireOption optionType, bool expiryExists, long n return (0, 0); } - internal static bool CheckExpiry(ref SpanByte src) => src.ExtraMetadata < DateTimeOffset.UtcNow.Ticks; + internal static bool CheckExpiry(ref TSourceLogRecord srcLogRecord) + where TSourceLogRecord : ISourceLogRecord + => srcLogRecord.Info.HasExpiration && srcLogRecord.Expiration < DateTimeOffset.UtcNow.Ticks; - static bool InPlaceUpdateNumber(long val, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo, int valueOffset) + static bool InPlaceUpdateNumber(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, long val, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { var ndigits = NumUtils.CountDigits(val, out var isNegative); ndigits += isNegative ? 1 : 0; - if (ndigits > value.LengthWithoutMetadata - valueOffset) + if (!logRecord.TrySetValueLength(ndigits, ref sizeInfo)) return false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(ndigits + value.MetadataSize + valueOffset); - _ = NumUtils.WriteInt64(val, value.AsSpan(valueOffset)); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + var value = logRecord.ValueSpan; // To eliminate redundant length calculations getting to Value + _ = NumUtils.WriteInt64(val, value); Debug.Assert(output.IsSpanByte, "This code assumes it is called in-place and did not go pending"); - value.AsReadOnlySpan(valueOffset).CopyTo(output.SpanByte.AsSpan()); - output.SpanByte.Length = value.LengthWithoutMetadata - valueOffset; + value.CopyTo(output.SpanByte.Span); + output.SpanByte.Length = value.Length; return true; } - static bool InPlaceUpdateNumber(double val, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo, int valueOffset) + static bool InPlaceUpdateNumber(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, double val, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { var ndigits = NumUtils.CountCharsInDouble(val, out var _, out var _, out var _); - if (ndigits > value.LengthWithoutMetadata - valueOffset) + if (!logRecord.TrySetValueLength(ndigits, ref sizeInfo)) return false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(ndigits + value.MetadataSize + valueOffset); - _ = NumUtils.WriteDouble(val, value.AsSpan(valueOffset)); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + var value = logRecord.ValueSpan; // To reduce redundant length calculations getting to Value + _ = NumUtils.WriteDouble(val, value); Debug.Assert(output.IsSpanByte, "This code assumes it is called in-place and did not go pending"); - value.AsReadOnlySpan(valueOffset).CopyTo(output.SpanByte.AsSpan()); - output.SpanByte.Length = value.LengthWithoutMetadata - valueOffset; + value.CopyTo(output.SpanByte.Span); + output.SpanByte.Length = value.Length; return true; } - static bool TryInPlaceUpdateNumber(ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo, long input, int valueOffset) + static bool TryInPlaceUpdateNumber(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, long input) { // Check if value contains a valid number - int valLen = value.LengthWithoutMetadata - valueOffset; - byte* valPtr = value.ToPointer() + valueOffset; - if (!IsValidNumber(valLen, valPtr, output.SpanByte.AsSpan(), out var val)) - return true; + var value = logRecord.ValueSpan; // To reduce redundant length calculations getting to Value + + long val; + if (logRecord.IsPinnedValue) + { + if (!IsValidNumber(value.Length, logRecord.PinnedValuePointer, output.SpanByte.Span, out val)) + return true; + } + else + { + fixed(byte* valuePtr = value) + { + if (!IsValidNumber(value.Length, valuePtr, output.SpanByte.Span, out val)) + return true; + } + } try { @@ -468,71 +513,99 @@ static bool TryInPlaceUpdateNumber(ref SpanByte value, ref SpanByteAndMemory out } catch { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; return true; } - return InPlaceUpdateNumber(val, ref value, ref output, ref rmwInfo, ref recordInfo, valueOffset); + return InPlaceUpdateNumber(ref logRecord, ref sizeInfo, val, ref output, ref rmwInfo); } - static bool TryInPlaceUpdateNumber(ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo, double input, int valueOffset) + static bool TryInPlaceUpdateNumber(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, double input) { - // Check if value contains a valid number - int valLen = value.LengthWithoutMetadata - valueOffset; - byte* valPtr = value.ToPointer() + valueOffset; - if (!IsValidDouble(valLen, valPtr, output.SpanByte.AsSpan(), out var val)) - return true; + var value = logRecord.ValueSpan; // To reduce redundant length calculations getting to Value + + double val; + if (logRecord.IsPinnedValue) + { + if (!IsValidDouble(value.Length, logRecord.PinnedValuePointer, output.SpanByte.Span, out val)) + return true; + } + else + { + fixed (byte* valuePtr = value) + { + if (!IsValidDouble(value.Length, valuePtr, output.SpanByte.Span, out val)) + return true; + } + } + val += input; if (!double.IsFinite(val)) { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; return true; } - return InPlaceUpdateNumber(val, ref value, ref output, ref rmwInfo, ref recordInfo, valueOffset); + return InPlaceUpdateNumber(ref logRecord, ref sizeInfo, val, ref output, ref rmwInfo); } - static void CopyUpdateNumber(long next, ref SpanByte newValue, ref SpanByteAndMemory output, int etagIgnoredOffset) + static bool TryCopyUpdateNumber(long next, Span newValue, ref SpanByteAndMemory output) { - NumUtils.WriteInt64(next, newValue.AsSpan(etagIgnoredOffset)); - newValue.AsReadOnlySpan(etagIgnoredOffset).CopyTo(output.SpanByte.AsSpan()); - output.SpanByte.Length = newValue.LengthWithoutMetadata - etagIgnoredOffset; - } - - static void CopyUpdateNumber(double next, ref SpanByte newValue, ref SpanByteAndMemory output, int etagIgnoredOffset) - { - NumUtils.WriteDouble(next, newValue.AsSpan(etagIgnoredOffset)); - newValue.AsReadOnlySpan(etagIgnoredOffset).CopyTo(output.SpanByte.AsSpan()); - output.SpanByte.Length = newValue.LengthWithoutMetadata - etagIgnoredOffset; + if (NumUtils.WriteInt64(next, newValue) == 0) + return false; + newValue.CopyTo(output.SpanByte.Span); + output.SpanByte.Length = newValue.Length; + return true; } - static void CopyUpdateNumber(double next, ref SpanByte newValue, ref SpanByteAndMemory output) + static bool TryCopyUpdateNumber(double next, Span newValue, ref SpanByteAndMemory output) { - NumUtils.WriteDouble(next, newValue.AsSpan()); - newValue.AsReadOnlySpan().CopyTo(output.SpanByte.AsSpan()); - output.SpanByte.Length = newValue.LengthWithoutMetadata; + if (NumUtils.WriteDouble(next, newValue) == 0) + return false; + newValue.CopyTo(output.SpanByte.Span); + output.SpanByte.Length = newValue.Length; + return true; } /// - /// Copy update from old value to new value while also validating whether oldValue is a numerical value. + /// Copy update from old 'long' value to new value while also validating whether oldValue is a numerical value. /// - /// Old value copying from - /// New value copying to + /// The source log record, either in-memory or from disk + /// The destination log record + /// Size info for record fields /// Output value /// Parsed input value - static void TryCopyUpdateNumber(ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, long input, int etagIgnoredOffset) + static bool TryCopyUpdateNumber(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref SpanByteAndMemory output, long input) + where TSourceLogRecord : ISourceLogRecord { - newValue.ExtraMetadata = oldValue.ExtraMetadata; + if (!dstLogRecord.TryCopyOptionals(ref srcLogRecord, ref sizeInfo)) + return false; - // Check if value contains a valid number - if (!IsValidNumber(oldValue.LengthWithoutMetadata - etagIgnoredOffset, oldValue.ToPointer() + etagIgnoredOffset, output.SpanByte.AsSpan(), out var val)) + var srcValue = srcLogRecord.ValueSpan; // To reduce redundant length calculations getting to ValueSpan + + long val; + if (srcLogRecord.IsPinnedValue) { - // Move to tail of the log even when oldValue is alphanumeric - // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory - oldValue.CopyTo(ref newValue); - return; + if (!IsValidNumber(srcValue.Length, srcLogRecord.PinnedValuePointer, output.SpanByte.Span, out val)) + { + // Move to tail of the log even when oldValue is alphanumeric + // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory + return dstLogRecord.TrySetValueSpan(srcLogRecord.ValueSpan, ref sizeInfo); + } + } + else + { + fixed (byte* valuePtr = srcValue) + { + if (!IsValidNumber(srcValue.Length, valuePtr, output.SpanByte.Span, out val)) + { + // Move to tail of the log even when oldValue is alphanumeric + // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory + return dstLogRecord.TrySetValueSpan(srcLogRecord.ValueSpan, ref sizeInfo); + } + } } // Check operation overflow @@ -542,44 +615,62 @@ static void TryCopyUpdateNumber(ref SpanByte oldValue, ref SpanByte newValue, re } catch { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; - return; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; + return false; } // Move to tail of the log and update - CopyUpdateNumber(val, ref newValue, ref output, etagIgnoredOffset); + return TryCopyUpdateNumber(val, dstLogRecord.ValueSpan, ref output); } /// - /// Copy update from old value to new value while also validating whether oldValue is a numerical value. + /// Copy update from old 'double' value to new value while also validating whether oldValue is a numerical value. /// - /// Old value copying from - /// New value copying to + /// The source log record, either in-memory or from disk + /// The destination log record + /// Size information for record fields /// Output value /// Parsed input value - /// Number of bytes to skip for ignoring etag in value payload - static void TryCopyUpdateNumber(ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, double input, int etagIgnoredOffset) + static bool TryCopyUpdateNumber(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref SpanByteAndMemory output, double input) + where TSourceLogRecord : ISourceLogRecord { - newValue.ExtraMetadata = oldValue.ExtraMetadata; + if (!dstLogRecord.TryCopyOptionals(ref srcLogRecord, ref sizeInfo)) + return false; - // Check if value contains a valid number - if (!IsValidDouble(oldValue.LengthWithoutMetadata - etagIgnoredOffset, oldValue.ToPointer() + etagIgnoredOffset, output.SpanByte.AsSpan(), out var val)) + var srcValue = srcLogRecord.ValueSpan; // To reduce redundant length calculations getting to ValueSpan + + double val; + if (srcLogRecord.IsPinnedValue) { - // Move to tail of the log even when oldValue is alphanumeric - // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory - oldValue.CopyTo(ref newValue); - return; + if (!IsValidDouble(srcValue.Length, srcLogRecord.PinnedValuePointer, output.SpanByte.Span, out val)) + { + // Move to tail of the log even when oldValue is alphanumeric + // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory + return dstLogRecord.TrySetValueSpan(srcLogRecord.ValueSpan, ref sizeInfo); + } + } + else + { + fixed (byte* valuePtr = srcValue) + { + if (!IsValidDouble(srcValue.Length, valuePtr, output.SpanByte.Span, out val)) + { + // Move to tail of the log even when oldValue is alphanumeric + // We have already paid the cost of bringing from disk so we are treating as a regular access and bring it into memory + return dstLogRecord.TrySetValueSpan(srcLogRecord.ValueSpan, ref sizeInfo); + } + } } val += input; if (!double.IsFinite(val)) { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; - return; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; + return false; } // Move to tail of the log and update - CopyUpdateNumber(val, ref newValue, ref output, etagIgnoredOffset); + return TryCopyUpdateNumber(val, dstLogRecord.ValueSpan, ref output); } /// @@ -614,82 +705,43 @@ static bool IsValidDouble(int length, byte* source, Span output, out doubl return true; } - void CopyDefaultResp(ReadOnlySpan resp, ref SpanByteAndMemory dst) - { - if (resp.Length < dst.SpanByte.Length) - { - resp.CopyTo(dst.SpanByte.AsSpan()); - dst.SpanByte.Length = resp.Length; - return; - } - - dst.ConvertToHeap(); - dst.Length = resp.Length; - dst.Memory = functionsState.memoryPool.Rent(resp.Length); - resp.CopyTo(dst.Memory.Memory.Span); - } - - void CopyRespNumber(long number, ref SpanByteAndMemory dst) - { - byte* curr = dst.SpanByte.ToPointer(); - byte* end = curr + dst.SpanByte.Length; - if (RespWriteUtils.TryWriteInt64(number, ref curr, end, out int integerLen, out int totalLen)) - { - dst.SpanByte.Length = (int)(curr - dst.SpanByte.ToPointer()); - return; - } - - //handle resp buffer overflow here - dst.ConvertToHeap(); - dst.Length = totalLen; - dst.Memory = functionsState.memoryPool.Rent(totalLen); - fixed (byte* ptr = dst.Memory.Memory.Span) - { - byte* cc = ptr; - *cc++ = (byte)':'; - NumUtils.WriteInt64(number, integerLen, ref cc); - *cc++ = (byte)'\r'; - *cc++ = (byte)'\n'; - } - } - /// /// Copy length of value to output (as ASCII bytes) /// - static void CopyValueLengthToOutput(ref SpanByte value, ref SpanByteAndMemory output, int eTagIgnoredOffset) + static bool CopyValueLengthToOutput(ReadOnlySpan value, ref SpanByteAndMemory output) { - int numDigits = NumUtils.CountDigits(value.LengthWithoutMetadata - eTagIgnoredOffset); - Debug.Assert(output.IsSpanByte, "This code assumes it is called in a non-pending context or in a pending context where dst.SpanByte's pointer remains valid"); + + var numDigits = NumUtils.CountDigits(value.Length); + if (numDigits > output.SpanByte.Length) + { + Debug.Fail("Output length overflow in CopyValueLengthToOutput"); + return false; + } + var outputPtr = output.SpanByte.ToPointer(); - NumUtils.WriteInt32(value.LengthWithoutMetadata - eTagIgnoredOffset, numDigits, ref outputPtr); + NumUtils.WriteInt32(value.Length, numDigits, ref outputPtr); output.SpanByte.Length = numDigits; + return true; } - static void CopyRespWithEtagData(ref SpanByte value, ref SpanByteAndMemory dst, bool hasEtagInVal, int etagSkippedStart, MemoryPool memoryPool) + void CopyRespWithEtagData(ReadOnlySpan value, ref SpanByteAndMemory dst, bool hasETag, MemoryPool memoryPool) { - int valueLength = value.LengthWithoutMetadata; + int valueLength = value.Length; // always writing an array of size 2 => *2\r\n int desiredLength = 4; - ReadOnlySpan etagTruncatedVal; + // get etag to write, default etag 0 for when no etag - long etag = hasEtagInVal ? value.GetEtagInPayload() : EtagConstants.NoETag; - // remove the length of the ETAG - var etagAccountedValueLength = valueLength - etagSkippedStart; - if (hasEtagInVal) - { - etagAccountedValueLength = valueLength - EtagConstants.EtagSize; - } + long etag = hasETag ? functionsState.etagState.ETag : LogRecord.NoETag; // here we know the value span has first bytes set to etag so we hardcode skipping past the bytes for the etag below - etagTruncatedVal = value.AsReadOnlySpan(etagSkippedStart); // *2\r\n :(etag digits)\r\n $(val Len digits)\r\n (value len)\r\n - desiredLength += 1 + NumUtils.CountDigits(etag) + 2 + 1 + NumUtils.CountDigits(etagAccountedValueLength) + 2 + etagAccountedValueLength + 2; + desiredLength += 1 + NumUtils.CountDigits(etag) + 2 + 1 + NumUtils.CountDigits(valueLength) + 2 + valueLength + 2; - WriteValAndEtagToDst(desiredLength, ref etagTruncatedVal, etag, ref dst, memoryPool); + WriteValAndEtagToDst(desiredLength, value, etag, ref dst, memoryPool); } - static void WriteValAndEtagToDst(int desiredLength, ref ReadOnlySpan value, long etag, ref SpanByteAndMemory dst, MemoryPool memoryPool, bool writeDirect = false) + static void WriteValAndEtagToDst(int desiredLength, ReadOnlySpan value, long etag, ref SpanByteAndMemory dst, MemoryPool memoryPool, bool writeDirect = false) { if (desiredLength <= dst.Length) { @@ -703,7 +755,7 @@ static void WriteValAndEtagToDst(int desiredLength, ref ReadOnlySpan value dst.ConvertToHeap(); dst.Length = desiredLength; dst.Memory = memoryPool.Rent(desiredLength); - fixed (byte* ptr = dst.Memory.Memory.Span) + fixed (byte* ptr = dst.MemorySpan) { byte* curr = ptr; byte* end = ptr + desiredLength; @@ -713,10 +765,10 @@ static void WriteValAndEtagToDst(int desiredLength, ref ReadOnlySpan value /// /// Logging upsert from - /// a. ConcurrentWriter - /// b. PostSingleWriter + /// a. InPlaceWriter + /// b. PostInitialWriter /// - void WriteLogUpsert(ref SpanByte key, ref RawStringInput input, ref SpanByte value, long version, int sessionId) + void WriteLogUpsert(ReadOnlySpan key, ref RawStringInput input, ReadOnlySpan value, long version, int sessionId) { if (functionsState.StoredProcMode) return; @@ -728,7 +780,7 @@ void WriteLogUpsert(ref SpanByte key, ref RawStringInput input, ref SpanByte val functionsState.appendOnlyFile.Enqueue( new AofHeader { opType = AofEntryType.StoreUpsert, storeVersion = version, sessionID = sessionId }, - ref key, ref value, ref input, out _); + key, value, ref input, out _); } /// @@ -737,26 +789,25 @@ void WriteLogUpsert(ref SpanByte key, ref RawStringInput input, ref SpanByte val /// b. InPlaceUpdater /// c. PostCopyUpdater /// - void WriteLogRMW(ref SpanByte key, ref RawStringInput input, long version, int sessionId) + void WriteLogRMW(ReadOnlySpan key, ref RawStringInput input, long version, int sessionId) { if (functionsState.StoredProcMode) return; input.header.flags |= RespInputFlags.Deterministic; functionsState.appendOnlyFile.Enqueue( new AofHeader { opType = AofEntryType.StoreRMW, storeVersion = version, sessionID = sessionId }, - ref key, ref input, out _); + key, ref input, out _); } /// /// Logging Delete from - /// a. ConcurrentDeleter - /// b. PostSingleDeleter + /// a. InPlaceDeleter + /// b. PostInitialDeleter /// - void WriteLogDelete(ref SpanByte key, long version, int sessionID) + void WriteLogDelete(ReadOnlySpan key, long version, int sessionID) { if (functionsState.StoredProcMode) return; - SpanByte def = default; - functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.StoreDelete, storeVersion = version, sessionID = sessionID }, ref key, ref def, out _); + functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.StoreDelete, storeVersion = version, sessionID = sessionID }, key, item2: default, out _); } BitFieldCmdArgs GetBitFieldArguments(ref RawStringInput input) diff --git a/libs/server/Storage/Functions/MainStore/RMWMethods.cs b/libs/server/Storage/Functions/MainStore/RMWMethods.cs index 4b6e0df350e..ab79ea43b98 100644 --- a/libs/server/Storage/Functions/MainStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/MainStore/RMWMethods.cs @@ -5,6 +5,7 @@ using System.Buffers; using System.Diagnostics; using Garnet.common; +using Microsoft.Extensions.Logging; using Tsavorite.core; namespace Garnet.server @@ -12,10 +13,10 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// - public bool NeedInitialUpdate(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public readonly bool NeedInitialUpdate(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { switch (input.header.cmd) { @@ -31,31 +32,25 @@ public bool NeedInitialUpdate(ref SpanByte key, ref RawStringInput input, ref Sp return false; case RespCommand.SETEXXX: // when called withetag all output needs to be placed on the buffer - if (input.header.CheckWithEtagFlag()) + if (input.header.CheckWithETagFlag()) { // XX when unsuccesful will write back NIL - CopyDefaultResp(functionsState.nilResp, ref output); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); } return false; case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - // add etag on first insertion - this.functionsState.etagState.etagOffsetForVarlen = EtagConstants.EtagSize; - return true; + // add etag on first insertion, already tracked by header.CheckWithEtagFlag() case RespCommand.SET: case RespCommand.SETEXNX: case RespCommand.SETKEEPTTL: - if (input.header.CheckWithEtagFlag()) - { - this.functionsState.etagState.etagOffsetForVarlen = EtagConstants.EtagSize; - } return true; default: if (input.header.cmd > RespCommandExtensions.LastValidCommand) { (IMemoryOwner Memory, int Length) outp = (output.Memory, 0); var ret = functionsState.GetCustomCommandFunctions((ushort)input.header.cmd) - .NeedInitialUpdate(key.AsReadOnlySpan(), ref input, ref outp); + .NeedInitialUpdate(key, ref input, ref outp); output.Memory = outp.Memory; output.Length = outp.Length; return ret; @@ -66,59 +61,73 @@ public bool NeedInitialUpdate(ref SpanByte key, ref RawStringInput input, ref Sp } /// - public bool InitialUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public readonly bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); + Debug.Assert(!logRecord.Info.HasETag && !logRecord.Info.HasExpiration, "Should not have Expiration or ETag on InitialUpdater log records"); + // Because this is InitialUpdater, the destination length should be set correctly, but test and log failures to be safe. RespCommand cmd = input.header.cmd; switch (cmd) { case RespCommand.PFADD: - var v = value.ToPointer(); - value.ShrinkSerializedLength(HyperLogLog.DefaultHLL.SparseInitialLength(ref input)); - HyperLogLog.DefaultHLL.Init(ref input, v, value.Length); + RecordSizeInfo.AssertValueDataLength(HyperLogLog.DefaultHLL.SparseInitialLength(ref input), ref sizeInfo); + if (!logRecord.TrySetValueLength(ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "PFADD"); + return false; + } + + var value = logRecord.ValueSpan; + if (logRecord.IsPinnedValue) + HyperLogLog.DefaultHLL.Init(ref input, logRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + HyperLogLog.DefaultHLL.Init(ref input, valuePtr, value.Length); + *output.SpanByte.ToPointer() = 1; break; case RespCommand.PFMERGE: //srcHLL offset: [hll allocated size = 4 byte] + [hll data structure] //memcpy + 4 (skip len size) - var sbSrcHLL = input.parseState.GetArgSliceByRef(0).SpanByte; - var length = sbSrcHLL.Length; - var srcHLL = sbSrcHLL.ToPointer(); - var dstHLL = value.ToPointer(); + var sbSrcHLL = input.parseState.GetArgSliceByRef(0); + + if (!logRecord.TrySetValueLength(sbSrcHLL.Length, ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "PFMERGE"); + return false; + } + + value = logRecord.ValueSpan; + + if (logRecord.IsPinnedValue) + Buffer.MemoryCopy(sbSrcHLL.ToPointer(), logRecord.PinnedValuePointer, value.Length, value.Length); + else + fixed (byte* valuePtr = value) + Buffer.MemoryCopy(sbSrcHLL.ToPointer(), valuePtr, value.Length, value.Length); - value.ShrinkSerializedLength(length); - Buffer.MemoryCopy(srcHLL, dstHLL, value.Length, value.Length); break; case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - int spaceForEtag = this.functionsState.etagState.etagOffsetForVarlen; // Copy input to value var newInputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - var metadataSize = input.arg1 == 0 ? 0 : sizeof(long); - value.ShrinkSerializedLength(newInputValue.Length + metadataSize + spaceForEtag); - value.ExtraMetadata = input.arg1; - newInputValue.CopyTo(value.AsSpan(spaceForEtag)); - - long clientSentEtag = input.parseState.GetLong(1); - - if (cmd == RespCommand.SETIFMATCH) - clientSentEtag++; + if (!logRecord.TrySetValueSpan(newInputValue, ref sizeInfo)) + return false; + if (sizeInfo.FieldInfo.HasExpiration) + _ = logRecord.TrySetExpiration(input.arg1); - recordInfo.SetHasETag(); // the increment on initial etag is for satisfying the variant that any key with no etag is the same as a zero'd etag - value.SetEtagInPayload(clientSentEtag); - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); + Debug.Assert(sizeInfo.FieldInfo.HasETag, "Expected sizeInfo.FieldInfo.HasETag to be true"); + _ = logRecord.TrySetETag(input.parseState.GetLong(1) + (cmd == RespCommand.SETIFMATCH ? 1 : 0)); + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref logRecord); // write back array of the format [etag, nil] var nilResponse = functionsState.nilResp; // *2\r\n: + + \r\n + WriteValAndEtagToDst( - 4 + 1 + NumUtils.CountDigits(functionsState.etagState.etag) + 2 + nilResponse.Length, - ref nilResponse, - functionsState.etagState.etag, + 4 + 1 + NumUtils.CountDigits(functionsState.etagState.ETag) + 2 + nilResponse.Length, + nilResponse, + functionsState.etagState.ETag, ref output, functionsState.memoryPool, writeDirect: true @@ -127,41 +136,44 @@ public bool InitialUpdater(ref SpanByte key, ref RawStringInput input, ref SpanB break; case RespCommand.SET: case RespCommand.SETEXNX: - spaceForEtag = this.functionsState.etagState.etagOffsetForVarlen; - // Copy input to value newInputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - metadataSize = input.arg1 == 0 ? 0 : sizeof(long); - value.ShrinkSerializedLength(newInputValue.Length + metadataSize + spaceForEtag); - value.ExtraMetadata = input.arg1; - newInputValue.CopyTo(value.AsSpan(spaceForEtag)); + if (!logRecord.TrySetValueSpan(newInputValue, ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "SETEXNX"); + return false; + } + + // the increment on initial etag is for satisfying the variant that any key with no etag is the same as a zero'd etag + if (sizeInfo.FieldInfo.HasETag && !logRecord.TrySetETag(LogRecord.NoETag + 1)) + { + functionsState.logger?.LogError("Could not set etag in {methodName}.{caseName}", "InitialUpdater", "SETEXNX"); + return false; + } + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref logRecord); + // Copy initial etag to output only for SET + WITHETAG and not SET NX or XX. TODO: Is this condition satisfied here? + functionsState.CopyRespNumber(LogRecord.NoETag + 1, ref output); - if (spaceForEtag != 0) + // Set or remove expiration + if (sizeInfo.FieldInfo.HasExpiration && !logRecord.TrySetExpiration(input.arg1)) { - recordInfo.SetHasETag(); - // the increment on initial etag is for satisfying the variant that any key with no etag is the same as a zero'd etag - value.SetEtagInPayload(EtagConstants.NoETag + 1); - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); - // Copy initial etag to output only for SET + WITHETAG and not SET NX or XX - CopyRespNumber(EtagConstants.NoETag + 1, ref output); + functionsState.logger?.LogError("Could not set expiration in {methodName}.{caseName}", "InitialUpdater", "SETEXNX"); + return false; } break; case RespCommand.SETKEEPTTL: - spaceForEtag = this.functionsState.etagState.etagOffsetForVarlen; - // Copy input to value - var setValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - value.ShrinkSerializedLength(value.MetadataSize + setValue.Length + spaceForEtag); - setValue.CopyTo(value.AsSpan(spaceForEtag)); + // Copy input to value; do not change expiration + _ = logRecord.TrySetValueSpan(input.parseState.GetArgSliceByRef(0).ReadOnlySpan, ref sizeInfo); - if (spaceForEtag != 0) + // the increment on initial etag is for satisfying the variant that any key with no etag is the same as a zero'd etag + if (sizeInfo.FieldInfo.HasETag && !logRecord.TrySetETag(LogRecord.NoETag + 1)) { - recordInfo.SetHasETag(); - value.SetEtagInPayload(EtagConstants.NoETag + 1); - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); - // Copy initial etag to output - CopyRespNumber(EtagConstants.NoETag + 1, ref output); + functionsState.logger?.LogError("Could not set etag in {methodName}.{caseName}", "InitialUpdater", "SETKEEPTTL"); + return false; } - + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref logRecord); + // Copy initial etag to output + functionsState.CopyRespNumber(LogRecord.NoETag + 1, ref output); break; case RespCommand.SETKEEPTTLXX: @@ -179,163 +191,233 @@ public bool InitialUpdater(ref SpanByte key, ref RawStringInput input, ref SpanB var bOffset = input.arg1; var bSetVal = (byte)(input.parseState.GetArgSliceByRef(1).ReadOnlySpan[0] - '0'); - value.ShrinkSerializedLength(BitmapManager.Length(bOffset)); + if (!logRecord.TrySetValueLength(BitmapManager.Length(bOffset), ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "SETBIT"); + return false; + } // Always return 0 at initial updater because previous value was 0 - BitmapManager.UpdateBitmap(value.ToPointer(), bOffset, bSetVal); - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); + value = logRecord.ValueSpan; + + if (logRecord.IsPinnedValue) + _ = BitmapManager.UpdateBitmap(logRecord.PinnedValuePointer, bOffset, bSetVal); + else + fixed (byte* valuePtr = value) + _ = BitmapManager.UpdateBitmap(valuePtr, bOffset, bSetVal); + + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); break; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - value.ShrinkSerializedLength(BitmapManager.LengthFromType(bitFieldArgs)); - var (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, value.ToPointer(), value.Length); + + if (!logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "BitField"); + return false; + } + + value = logRecord.ValueSpan; + + long bitfieldReturnValue; + bool overflow; + if (logRecord.IsPinnedValue) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, logRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, valuePtr, value.Length); + if (!overflow) - CopyRespNumber(bitfieldReturnValue, ref output); + functionsState.CopyRespNumber(bitfieldReturnValue, ref output); else - CopyDefaultResp(functionsState.nilResp, ref output); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); break; case RespCommand.BITFIELD_RO: var bitFieldArgs_RO = GetBitFieldArguments(ref input); - value.ShrinkSerializedLength(BitmapManager.LengthFromType(bitFieldArgs_RO)); - var bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, value.ToPointer(), value.Length); - CopyRespNumber(bitfieldReturnValue_RO, ref output); + if (!logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs_RO), ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "BitField"); + return false; + } + value = logRecord.ValueSpan; + + long bitfieldReturnValue_RO; + if (logRecord.IsPinnedValue) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, logRecord.PinnedValuePointer, value.Length); + else + fixed (byte* valuePtr = value) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, valuePtr, value.Length); + + functionsState.CopyRespNumber(bitfieldReturnValue_RO, ref output); break; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - newValue.CopyTo(value.AsSpan().Slice(offset)); - CopyValueLengthToOutput(ref value, ref output, 0); + value = logRecord.ValueSpan; + newValue.CopyTo(value.Slice(offset)); + + if (!CopyValueLengthToOutput(value, ref output)) + return false; break; case RespCommand.APPEND: var appendValue = input.parseState.GetArgSliceByRef(0); - value.ShrinkSerializedLength(appendValue.Length); // Copy value to be appended to the newly allocated value buffer - appendValue.ReadOnlySpan.CopyTo(value.AsSpan()); + value = logRecord.ValueSpan; + appendValue.ReadOnlySpan.CopyTo(value); - CopyValueLengthToOutput(ref value, ref output, 0); + if (!CopyValueLengthToOutput(value, ref output)) + return false; break; case RespCommand.INCR: - value.ShrinkSerializedLength(1); // # of digits in "1" - CopyUpdateNumber(1, ref value, ref output); + // This is InitialUpdater so set the value to 1 and the length to the # of digits in "1" + if (!logRecord.TrySetValueLength(1, ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "INCR"); + return false; + } + + value = logRecord.ValueSpan; + _ = TryCopyUpdateNumber(1L, value, ref output); break; case RespCommand.INCRBY: var incrBy = input.arg1; + var ndigits = NumUtils.CountDigits(incrBy, out var isNegative); - value.ShrinkSerializedLength(ndigits + (isNegative ? 1 : 0)); - CopyUpdateNumber(incrBy, ref value, ref output); + if (!logRecord.TrySetValueLength(ndigits + (isNegative ? 1 : 0), ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "INCRBY"); + return false; + } + + _ = TryCopyUpdateNumber(incrBy, logRecord.ValueSpan, ref output); break; case RespCommand.DECR: - value.ShrinkSerializedLength(2); // # of digits in "-1" - CopyUpdateNumber(-1, ref value, ref output); + // This is InitialUpdater so set the value to -1 and the length to the # of digits in "-1" + if (!logRecord.TrySetValueLength(2, ref sizeInfo)) + { + Debug.Assert(logRecord.ValueSpan.Length >= 2, "Length overflow in DECR"); + return false; + } + value = logRecord.ValueSpan; + _ = TryCopyUpdateNumber(-1, value, ref output); break; case RespCommand.DECRBY: - isNegative = false; var decrBy = -input.arg1; + ndigits = NumUtils.CountDigits(decrBy, out isNegative); - value.ShrinkSerializedLength(ndigits + (isNegative ? 1 : 0)); - CopyUpdateNumber(decrBy, ref value, ref output); + if (!logRecord.TrySetValueLength(ndigits + (isNegative ? 1 : 0), ref sizeInfo)) + { + functionsState.logger?.LogError("Length overflow in {methodName}.{caseName}", "InitialUpdater", "DECRBY"); + return false; + } + + _ = TryCopyUpdateNumber(decrBy, logRecord.ValueSpan, ref output); break; case RespCommand.INCRBYFLOAT: // Check if input contains a valid number if (!input.parseState.TryGetDouble(0, out var incrByFloat)) { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; return true; } - CopyUpdateNumber(incrByFloat, ref value, ref output); + + value = logRecord.ValueSpan; + if (!TryCopyUpdateNumber(incrByFloat, value, ref output)) + return false; break; default: if (input.header.cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)input.header.cmd); - // compute metadata size for result - var expiration = input.arg1; - metadataSize = expiration switch + if (!logRecord.TrySetValueLength(functions.GetInitialLength(ref input), ref sizeInfo)) { - -1 => 0, - 0 => 0, - _ => 8, - }; - - value.ShrinkSerializedLength(metadataSize + functions.GetInitialLength(ref input)); - if (expiration > 0) - value.ExtraMetadata = expiration; + functionsState.logger?.LogError("Length overflow in 'default' > StartOffset: {methodName}.{caseName}", "InitialUpdater", "default"); + return false; + } + if (input.arg1 > 0 && !logRecord.TrySetExpiration(input.arg1)) + { + functionsState.logger?.LogError("Could not set expiration in 'default' > StartOffset: {methodName}.{caseName}", "InitialUpdater", "default"); + return false; + } (IMemoryOwner Memory, int Length) outp = (output.Memory, 0); - functions.InitialUpdater(key.AsReadOnlySpan(), ref input, value.AsSpan(), ref outp, ref rmwInfo); + value = logRecord.ValueSpan; + if (!functions.InitialUpdater(logRecord.Key, ref input, value, ref outp, ref rmwInfo)) + return false; output.Memory = outp.Memory; output.Length = outp.Length; break; } // Copy input to value - var inputValue = input.parseState.GetArgSliceByRef(0); - value.ShrinkSerializedLength(inputValue.Length); - value.ExtraMetadata = input.arg1; - inputValue.ReadOnlySpan.CopyTo(value.AsSpan()); + if (!logRecord.TrySetValueSpan(input.parseState.GetArgSliceByRef(0).ReadOnlySpan, ref sizeInfo)) + { + functionsState.logger?.LogError("Failed to set value in {methodName}.{caseName}", "InitialUpdater", "default"); + return false; + } // Copy value to output - CopyTo(ref value, ref output, functionsState.memoryPool); + CopyTo(logRecord.ValueSpan, ref output, functionsState.memoryPool); break; } - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + // Success if we made it here + sizeInfo.AssertOptionals(logRecord.Info); return true; } /// - public void PostInitialUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public readonly void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { // reset etag state set at need initial update if (input.header.cmd is (RespCommand.SET or RespCommand.SETEXNX or RespCommand.SETKEEPTTL or RespCommand.SETIFMATCH or RespCommand.SETIFGREATER)) - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); if (functionsState.appendOnlyFile != null) { input.header.SetExpiredFlag(); - WriteLogRMW(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + WriteLogRMW(logRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); } } /// - public bool InPlaceUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public readonly bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - if (InPlaceUpdaterWorker(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo)) + if (InPlaceUpdaterWorker(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) { - rmwInfo.UsedValueLength = value.TotalSize; - if (!rmwInfo.RecordInfo.Modified) + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogRMW(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + WriteLogRMW(logRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); return true; } return false; } - // NOTE: In the below control flow if you decide to add a new command or modify a command such that it will now do an early return with TRUE, you must make sure you must reset etagState in FunctionState - private bool InPlaceUpdaterWorker(ref SpanByte key, ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + // NOTE: In the below control flow if you decide to add a new command or modify a command such that it will now do an early return with TRUE, + // you must make sure you must reset etagState in FunctionState + private readonly bool InPlaceUpdaterWorker(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { // Expired data - if (value.MetadataSize > 0 && input.header.CheckExpiry(value.ExtraMetadata)) + if (logRecord.Info.HasExpiration && input.header.CheckExpiry(logRecord.Expiration)) { rmwInfo.Action = RMWAction.ExpireAndResume; - recordInfo.ClearHasETag(); + logRecord.RemoveETag(); return false; } RespCommand cmd = input.header.cmd; - bool hadRecordPreMutation = recordInfo.ETag; - bool shouldUpdateEtag = hadRecordPreMutation; + bool hadETagPreMutation = logRecord.Info.HasETag; + bool shouldUpdateEtag = hadETagPreMutation; if (shouldUpdateEtag) - { - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); - } + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref logRecord); + bool shouldCheckExpiration = true; switch (cmd) { @@ -343,243 +425,142 @@ private bool InPlaceUpdaterWorker(ref SpanByte key, ref RawStringInput input, re if (input.header.CheckSetGetFlag()) { // Copy value to output for the GET part of the command. - CopyRespTo(ref value, ref output); + CopyRespTo(logRecord.ValueSpan, ref output); } - else if (input.header.CheckWithEtagFlag()) + else if (input.header.CheckWithETagFlag()) { // when called withetag all output needs to be placed on the buffer // EXX when unsuccesful will write back NIL - CopyDefaultResp(functionsState.nilResp, ref output); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); } // reset etag state after done using - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); // Nothing is set because being in this block means NX was already violated return true; case RespCommand.DELIFGREATER: long etagFromClient = input.parseState.GetLong(0); - rmwInfo.Action = etagFromClient > functionsState.etagState.etag ? RMWAction.ExpireAndStop : RMWAction.CancelOperation; - EtagState.ResetState(ref functionsState.etagState); + rmwInfo.Action = etagFromClient > functionsState.etagState.ETag ? RMWAction.ExpireAndStop : RMWAction.CancelOperation; + ETagState.ResetState(ref functionsState.etagState); return false; case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: etagFromClient = input.parseState.GetLong(1); // in IFMATCH we check for equality, in IFGREATER we are checking for sent etag being strictly greater - int comparisonResult = etagFromClient.CompareTo(functionsState.etagState.etag); + int comparisonResult = etagFromClient.CompareTo(functionsState.etagState.ETag); int expectedResult = cmd is RespCommand.SETIFMATCH ? 0 : 1; if (comparisonResult != expectedResult) { if (input.header.CheckSetGetFlag()) - { - CopyRespWithEtagData(ref value, ref output, shouldUpdateEtag, functionsState.etagState.etagSkippedStart, functionsState.memoryPool); - } + CopyRespWithEtagData(logRecord.ValueSpan, ref output, shouldUpdateEtag, functionsState.memoryPool); else { // write back array of the format [etag, nil] var nilResponse = functionsState.nilResp; // *2\r\n: + + \r\n + WriteValAndEtagToDst( - 4 + 1 + NumUtils.CountDigits(functionsState.etagState.etag) + 2 + nilResponse.Length, - ref nilResponse, - functionsState.etagState.etag, + 4 + 1 + NumUtils.CountDigits(functionsState.etagState.ETag) + 2 + nilResponse.Length, + nilResponse, + functionsState.etagState.ETag, ref output, functionsState.memoryPool, writeDirect: true ); } // reset etag state after done using - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return true; } - // Need Copy update if no space for new value - var inputValue = input.parseState.GetArgSliceByRef(0); - - // retain metadata unless metadata sent - int metadataSize = input.arg1 != 0 ? sizeof(long) : value.MetadataSize; - - if (value.Length < inputValue.length + EtagConstants.EtagSize + metadataSize) + // If we're here we know we have a valid ETag for update. Get the value to update. We'll ned to return false for CopyUpdate if no space for new value. + var inputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; + if (!logRecord.TrySetValueSpan(inputValue, ref sizeInfo)) + return false; + long newEtag = cmd is RespCommand.SETIFMATCH ? (functionsState.etagState.ETag + 1) : etagFromClient; + if (!logRecord.TrySetETag(newEtag)) return false; - recordInfo.SetHasETag(); - - long newEtag = cmd is RespCommand.SETIFMATCH ? (functionsState.etagState.etag + 1) : etagFromClient; - - long oldExtraMetadata = value.ExtraMetadata; - - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - value.ShrinkSerializedLength(metadataSize + inputValue.Length + EtagConstants.EtagSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - - if (input.arg1 != 0) - { - value.ExtraMetadata = input.arg1; - } - else if (oldExtraMetadata != 0) - { - value.ExtraMetadata = oldExtraMetadata; - } - - value.SetEtagInPayload(newEtag); - - inputValue.ReadOnlySpan.CopyTo(value.AsSpan(EtagConstants.EtagSize)); + // Need to check for input.arg1 != 0 because GetRMWModifiedFieldInfo shares its logic with CopyUpdater and thus may set sizeInfo.FieldInfo.Expiration true + // due to srcRecordInfo having expiration set; here, that srcRecordInfo is us, so we should do nothing if input.arg1 == 0. + if (sizeInfo.FieldInfo.HasExpiration && input.arg1 != 0 && !logRecord.TrySetExpiration(input.arg1)) + return false; - // write back array of the format [etag, nil] + // Write Etag and Val back to Client as an array of the format [etag, nil] var nilResp = functionsState.nilResp; // *2\r\n: + + \r\n + var numDigitsInEtag = NumUtils.CountDigits(newEtag); - WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + nilResp.Length, ref nilResp, newEtag, ref output, functionsState.memoryPool, writeDirect: true); + WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + nilResp.Length, nilResp, newEtag, ref output, functionsState.memoryPool, writeDirect: true); // reset etag state after done using - EtagState.ResetState(ref functionsState.etagState); - // early return since we already updated the ETag - return true; + ETagState.ResetState(ref functionsState.etagState); + shouldUpdateEtag = false; // since we already updated the ETag + break; case RespCommand.SET: case RespCommand.SETEXXX: - // If the user calls withetag then we need to either update an existing etag and set the value or set the value with an etag and increment it. - bool inputHeaderHasEtag = input.header.CheckWithEtagFlag(); - - int nextUpdateEtagOffset = functionsState.etagState.etagSkippedStart; - - // only when both are not false && false or true and true, do we need to readjust - if (inputHeaderHasEtag != shouldUpdateEtag) - { - // in the common path the above condition is skipped - if (inputHeaderHasEtag) - { - // nextUpdate will add etag but currently there is no etag - nextUpdateEtagOffset = EtagConstants.EtagSize; - shouldUpdateEtag = true; - // if something is going to go past this into copy we need to provide offset management for its varlen during allocation - this.functionsState.etagState.etagOffsetForVarlen = EtagConstants.EtagSize; - } - else - { - shouldUpdateEtag = false; - // nextUpdate will remove etag but currently there is an etag - nextUpdateEtagOffset = 0; - this.functionsState.etagState.etagOffsetForVarlen = 0; - } - } - - ArgSlice setValue = input.parseState.GetArgSliceByRef(0); - - // Need CU if no space for new value - metadataSize = input.arg1 == 0 ? 0 : sizeof(long); - if (setValue.Length + metadataSize > value.Length - nextUpdateEtagOffset) - return false; - // Check if SetGet flag is set if (input.header.CheckSetGetFlag()) { - Debug.Assert(!input.header.CheckWithEtagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); // Copy value to output for the GET part of the command. - CopyRespTo(ref value, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(logRecord.ValueSpan, ref output); } - // Adjust value length - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - value.ShrinkSerializedLength(setValue.Length + metadataSize + nextUpdateEtagOffset); + // If the user calls withetag then we need to either update an existing etag and set the value or set the value with an etag and increment it. + bool inputHeaderHasEtag = input.header.CheckWithETagFlag(); - // Copy input to value - value.ExtraMetadata = input.arg1; - setValue.ReadOnlySpan.CopyTo(value.AsSpan(nextUpdateEtagOffset)); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + var setValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; + if (!logRecord.TrySetValueSpan(setValue, ref sizeInfo)) + return false; - // If withEtag is called we return the etag back in the response + if (inputHeaderHasEtag != shouldUpdateEtag) + shouldUpdateEtag = inputHeaderHasEtag; if (inputHeaderHasEtag) { - recordInfo.SetHasETag(); - value.SetEtagInPayload(functionsState.etagState.etag + 1); - // withetag flag means we need to write etag back to the output buffer - CopyRespNumber(functionsState.etagState.etag + 1, ref output); - // reset etag state after done using - EtagState.ResetState(ref functionsState.etagState); - // early return since we already updated etag - return true; + var newETag = functionsState.etagState.ETag + 1; + logRecord.TrySetETag(newETag); + functionsState.CopyRespNumber(newETag, ref output); } else - { - recordInfo.ClearHasETag(); - } + logRecord.RemoveETag(); + shouldUpdateEtag = false; // since we already updated the ETag + if (!(input.arg1 == 0 ? logRecord.RemoveExpiration() : logRecord.TrySetExpiration(input.arg1))) + return false; break; case RespCommand.SETKEEPTTLXX: case RespCommand.SETKEEPTTL: // If the user calls withetag then we need to either update an existing etag and set the value - // or set the value with an initial etag and increment it. - // If withEtag is called we return the etag back to the user - inputHeaderHasEtag = input.header.CheckWithEtagFlag(); - - nextUpdateEtagOffset = functionsState.etagState.etagSkippedStart; - - // only when both are not false && false or true and true, do we need to readjust - if (inputHeaderHasEtag != shouldUpdateEtag) - { - // in the common path the above condition is skipped - if (inputHeaderHasEtag) - { - // nextUpdate will add etag but currently there is no etag - nextUpdateEtagOffset = EtagConstants.EtagSize; - shouldUpdateEtag = true; - // if something is going to go past this into copy we need to provide offset management for its varlen during allocation - this.functionsState.etagState.etagOffsetForVarlen = EtagConstants.EtagSize; - } - else - { - shouldUpdateEtag = false; - // nextUpdate will remove etag but currentyly there is an etag - nextUpdateEtagOffset = 0; - this.functionsState.etagState.etagOffsetForVarlen = 0; - } - } + // or set the value with an initial etag and increment it. If withEtag is called we return the etag back to the user + inputHeaderHasEtag = input.header.CheckWithETagFlag(); - setValue = input.parseState.GetArgSliceByRef(0); - // Need CU if no space for new value - if (setValue.Length + value.MetadataSize > value.Length - nextUpdateEtagOffset) - return false; - - // Check if SetGet flag is set + // If the SetGet flag is set, copy the current value to output for the GET part of the command. if (input.header.CheckSetGetFlag()) { - Debug.Assert(!input.header.CheckWithEtagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); + Debug.Assert(!input.header.CheckWithETagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); + // Copy value to output for the GET part of the command. - CopyRespTo(ref value, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(logRecord.ValueSpan, ref output); } - // Adjust value length - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(setValue.Length + value.MetadataSize + functionsState.etagState.etagSkippedStart); - - // Copy input to value - setValue.ReadOnlySpan.CopyTo(value.AsSpan(functionsState.etagState.etagSkippedStart)); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + setValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; + if (!logRecord.TrySetValueSpan(setValue, ref sizeInfo)) + return false; + if (inputHeaderHasEtag != shouldUpdateEtag) + shouldUpdateEtag = inputHeaderHasEtag; if (inputHeaderHasEtag) { - recordInfo.SetHasETag(); - value.SetEtagInPayload(functionsState.etagState.etag + 1); - // withetag flag means we need to write etag back to the output buffer - CopyRespNumber(functionsState.etagState.etag + 1, ref output); - // reset etag state after done using - EtagState.ResetState(ref functionsState.etagState); - // early return since we already updated etag - return true; + var newETag = functionsState.etagState.ETag + 1; + logRecord.TrySetETag(newETag); + functionsState.CopyRespNumber(newETag, ref output); } else - { - recordInfo.ClearHasETag(); - } - + logRecord.RemoveETag(); + shouldUpdateEtag = false; // since we already updated the ETag break; case RespCommand.PEXPIRE: case RespCommand.EXPIRE: - var expiryExists = value.MetadataSize > 0; - var expiryValue = input.parseState.GetLong(0); var tsExpiry = input.header.cmd == RespCommand.EXPIRE ? TimeSpan.FromSeconds(expiryValue) @@ -587,259 +568,304 @@ private bool InPlaceUpdaterWorker(ref SpanByte key, ref RawStringInput input, re var expiryTicks = DateTimeOffset.UtcNow.Ticks + tsExpiry.Ticks; var expireOption = (ExpireOption)input.arg1; - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + // 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; - if (!EvaluateExpireInPlace(expireOption, expiryExists, expiryTicks, ref value, ref output)) + if (!EvaluateExpireInPlace(ref logRecord, expireOption, expiryTicks, ref output)) return false; - - // doesn't update etag, since it's only the metadata that was updated - return true; + break; case RespCommand.PEXPIREAT: case RespCommand.EXPIREAT: - expiryExists = value.MetadataSize > 0; - var expiryTimestamp = input.parseState.GetLong(0); expiryTicks = input.header.cmd == RespCommand.PEXPIREAT ? ConvertUtils.UnixTimestampInMillisecondsToTicks(expiryTimestamp) : ConvertUtils.UnixTimestampInSecondsToTicks(expiryTimestamp); expireOption = (ExpireOption)input.arg1; - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + // 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; - if (!EvaluateExpireInPlace(expireOption, expiryExists, expiryTicks, ref value, ref output)) + if (!EvaluateExpireInPlace(ref logRecord, expireOption, expiryTicks, ref output)) return false; - - // doesn't update etag, since it's only the metadata that was updated - return true; + break; case RespCommand.PERSIST: - if (value.MetadataSize != 0) + if (logRecord.Info.HasExpiration) { - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.AsSpan().CopyTo(value.AsSpanWithMetadata()); - value.ShrinkSerializedLength(value.Length - value.MetadataSize); - value.UnmarkExtraMetadata(); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - output.SpanByte.AsSpan()[0] = 1; + _ = logRecord.RemoveExpiration(); + output.SpanByte.Span[0] = 1; } - // does not update etag - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - return true; + + // 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; + break; case RespCommand.INCR: - if (!TryInPlaceUpdateNumber(ref value, ref output, ref rmwInfo, ref recordInfo, input: 1, functionsState.etagState.etagSkippedStart)) + if (!TryInPlaceUpdateNumber(ref logRecord, ref sizeInfo, ref output, ref rmwInfo, input: 1)) return false; break; - case RespCommand.DECR: - if (!TryInPlaceUpdateNumber(ref value, ref output, ref rmwInfo, ref recordInfo, input: -1, functionsState.etagState.etagSkippedStart)) - { - + if (!TryInPlaceUpdateNumber(ref logRecord, ref sizeInfo, ref output, ref rmwInfo, input: -1)) return false; - } break; - case RespCommand.INCRBY: // Check if input contains a valid number var incrBy = input.arg1; - if (!TryInPlaceUpdateNumber(ref value, ref output, ref rmwInfo, ref recordInfo, input: incrBy, functionsState.etagState.etagSkippedStart)) + if (!TryInPlaceUpdateNumber(ref logRecord, ref sizeInfo, ref output, ref rmwInfo, input: incrBy)) return false; break; - case RespCommand.DECRBY: var decrBy = input.arg1; - if (!TryInPlaceUpdateNumber(ref value, ref output, ref rmwInfo, ref recordInfo, input: -decrBy, functionsState.etagState.etagSkippedStart)) + if (!TryInPlaceUpdateNumber(ref logRecord, ref sizeInfo, ref output, ref rmwInfo, input: -decrBy)) return false; break; - case RespCommand.INCRBYFLOAT: // Check if input contains a valid number if (!input.parseState.TryGetDouble(0, out var incrByFloat)) { - output.SpanByte.AsSpan()[0] = (byte)OperationError.INVALID_TYPE; - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - return true; + output.SpanByte.Span[0] = (byte)OperationError.INVALID_TYPE; + + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); + shouldUpdateEtag = false; + break; } - if (!TryInPlaceUpdateNumber(ref value, ref output, ref rmwInfo, ref recordInfo, incrByFloat, functionsState.etagState.etagSkippedStart)) + if (!TryInPlaceUpdateNumber(ref logRecord, ref sizeInfo, ref output, ref rmwInfo, incrByFloat)) return false; break; case RespCommand.SETBIT: - var v = value.ToPointer() + functionsState.etagState.etagSkippedStart; var bOffset = input.arg1; var bSetVal = (byte)(input.parseState.GetArgSliceByRef(1).ReadOnlySpan[0] - '0'); - if (!BitmapManager.IsLargeEnough(functionsState.etagState.etagAccountedLength, bOffset)) return false; + if (!BitmapManager.IsLargeEnough(logRecord.ValueSpan.Length, bOffset) + && !logRecord.TrySetValueLength(BitmapManager.Length(bOffset), ref sizeInfo)) + return false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - value.ShrinkSerializedLength(value.Length + value.MetadataSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + _ = logRecord.RemoveExpiration(); + + byte oldValSet; + if (logRecord.IsPinnedValue) + oldValSet = BitmapManager.UpdateBitmap(logRecord.PinnedValuePointer, bOffset, bSetVal); + else + fixed (byte* valuePtr = logRecord.ValueSpan) + oldValSet = BitmapManager.UpdateBitmap(valuePtr, bOffset, bSetVal); - var oldValSet = BitmapManager.UpdateBitmap(v, bOffset, bSetVal); if (oldValSet == 0) - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); else - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output); break; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - v = value.ToPointer() + functionsState.etagState.etagSkippedStart; - - if (!BitmapManager.IsLargeEnoughForType(bitFieldArgs, value.Length - functionsState.etagState.etagSkippedStart)) + if (!BitmapManager.IsLargeEnoughForType(bitFieldArgs, logRecord.ValueSpan.Length) + && !logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs), ref sizeInfo)) return false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - value.ShrinkSerializedLength(value.Length + value.MetadataSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + _ = logRecord.RemoveExpiration(); - var (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, v, value.Length - functionsState.etagState.etagSkippedStart); + long bitfieldReturnValue; + bool overflow; + if (logRecord.IsPinnedValue) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, logRecord.PinnedValuePointer, logRecord.ValueSpan.Length); + else + fixed (byte* valuePtr = logRecord.ValueSpan) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, valuePtr, logRecord.ValueSpan.Length); if (overflow) { - CopyDefaultResp(functionsState.nilResp, ref output); - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - // etag not updated + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); + + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); + shouldUpdateEtag = false; return true; } - CopyRespNumber(bitfieldReturnValue, ref output); + functionsState.CopyRespNumber(bitfieldReturnValue, ref output); break; + case RespCommand.BITFIELD_RO: var bitFieldArgs_RO = GetBitFieldArguments(ref input); - v = value.ToPointer() + functionsState.etagState.etagSkippedStart; - if (!BitmapManager.IsLargeEnoughForType(bitFieldArgs_RO, value.Length - functionsState.etagState.etagSkippedStart)) + if (!BitmapManager.IsLargeEnoughForType(bitFieldArgs_RO, logRecord.ValueSpan.Length) + && !logRecord.TrySetValueLength(BitmapManager.LengthFromType(bitFieldArgs_RO), ref sizeInfo)) return false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - value.ShrinkSerializedLength(value.Length + value.MetadataSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); + _ = logRecord.RemoveExpiration(); + + long bitfieldReturnValue_RO; + if (logRecord.IsPinnedValue) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, logRecord.PinnedValuePointer, logRecord.ValueSpan.Length); + else + fixed (byte* valuePtr = logRecord.ValueSpan) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, valuePtr, logRecord.ValueSpan.Length); - var bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, v, - value.Length - functionsState.etagState.etagSkippedStart); - CopyRespNumber(bitfieldReturnValue_RO, ref output); + functionsState.CopyRespNumber(bitfieldReturnValue_RO, ref output); break; case RespCommand.PFADD: - v = value.ToPointer(); + bool result = false, parseOk = false; + var updated = false; + var valueLen = logRecord.ValueSpan.Length; + if (logRecord.IsPinnedValue) + { + parseOk = result = HyperLogLog.DefaultHLL.IsValidHYLL(logRecord.PinnedValuePointer, valueLen); + if (result) + { + _ = logRecord.RemoveExpiration(); + result = HyperLogLog.DefaultHLL.Update(ref input, logRecord.PinnedValuePointer, valueLen, ref updated); + } + } + else + { + fixed (byte* valuePtr = logRecord.ValueSpan) + { + parseOk = result = HyperLogLog.DefaultHLL.IsValidHYLL(valuePtr, valueLen); + if (result) + { + _ = logRecord.RemoveExpiration(); + result = HyperLogLog.DefaultHLL.Update(ref input, valuePtr, valueLen, ref updated); + } + } + } - if (!HyperLogLog.DefaultHLL.IsValidHYLL(v, value.Length)) + if (!parseOk) { - *output.SpanByte.ToPointer() = (byte)0xFF; - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + *output.SpanByte.ToPointer() = (byte)0xFF; // Flags invalid HLL + + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); return true; } - var updated = false; - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(value.Length + value.MetadataSize); - var result = HyperLogLog.DefaultHLL.Update(ref input, v, value.Length, ref updated); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - if (result) *output.SpanByte.ToPointer() = updated ? (byte)1 : (byte)0; - return result; + if (!result) + return false; + break; case RespCommand.PFMERGE: //srcHLL offset: [hll allocated size = 4 byte] + [hll data structure] //memcpy +4 (skip len size) - var srcHLL = input.parseState.GetArgSliceByRef(0).SpanByte.ToPointer(); - var dstHLL = value.ToPointer(); + var srcHLL = input.parseState.GetArgSliceByRef(0).ToPointer(); - if (!HyperLogLog.DefaultHLL.IsValidHYLL(dstHLL, value.Length)) + result = parseOk = false; + valueLen = logRecord.ValueSpan.Length; + if (logRecord.IsPinnedValue) + { + var dstHLL = logRecord.PinnedValuePointer; + parseOk = result = HyperLogLog.DefaultHLL.IsValidHYLL(dstHLL, valueLen); + if (result) + { + _ = logRecord.RemoveExpiration(); + result = HyperLogLog.DefaultHLL.TryMerge(srcHLL, dstHLL, valueLen); + } + } + else + { + fixed (byte* dstHLL = logRecord.ValueSpan) + { + parseOk = result = HyperLogLog.DefaultHLL.IsValidHYLL(dstHLL, valueLen); + if (result) + { + _ = logRecord.RemoveExpiration(); + result = HyperLogLog.DefaultHLL.TryMerge(srcHLL, dstHLL, valueLen); + } + } + } + + if (!parseOk) { - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); //InvalidType - *(long*)output.SpanByte.ToPointer() = -1; + *output.SpanByte.ToPointer() = (byte)0xFF; // Flags invalid HLL + + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); return true; } - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(value.Length + value.MetadataSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - return HyperLogLog.DefaultHLL.TryMerge(srcHLL, dstHLL, value.Length); + if (!result) + return false; + break; + case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - if (newValue.Length + offset > value.LengthWithoutMetadata - functionsState.etagState.etagSkippedStart) + if (newValue.Length + offset > logRecord.ValueSpan.Length + && !logRecord.TrySetValueLength(newValue.Length + offset, ref sizeInfo)) return false; - newValue.CopyTo(value.AsSpan(functionsState.etagState.etagSkippedStart).Slice(offset)); - - CopyValueLengthToOutput(ref value, ref output, functionsState.etagState.etagSkippedStart); + newValue.CopyTo(logRecord.ValueSpan.Slice(offset)); + if (!CopyValueLengthToOutput(logRecord.ValueSpan, ref output)) + return false; break; case RespCommand.GETDEL: // Copy value to output for the GET part of the command. // Then, set ExpireAndStop action to delete the record. - CopyRespTo(ref value, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(logRecord.ValueSpan, ref output); rmwInfo.Action = RMWAction.ExpireAndStop; return false; case RespCommand.GETEX: - CopyRespTo(ref value, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(logRecord.ValueSpan, ref output); + // If both EX and PERSIST were specified, EX wins if (input.arg1 > 0) { - byte* pbOutput = stackalloc byte[ObjectOutputHeader.Size]; - var _output = new SpanByteAndMemory(SpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); + var pbOutput = stackalloc byte[ObjectOutputHeader.Size]; + var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); var newExpiry = input.arg1; - return EvaluateExpireInPlace(ExpireOption.None, expiryExists: value.MetadataSize > 0, newExpiry, ref value, ref _output); + if (!EvaluateExpireInPlace(ref logRecord, ExpireOption.None, newExpiry, ref _output)) + return false; } - - if (input.parseState.Count > 0) + else if (!sizeInfo.FieldInfo.HasExpiration) { - var persist = input.parseState.GetArgSliceByRef(0).ReadOnlySpan - .EqualsUpperCaseSpanIgnoringCase(CmdStrings.PERSIST); - - if (persist) // Persist the key - { - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.AsSpan().CopyTo(value.AsSpanWithMetadata()); - value.ShrinkSerializedLength(value.Length - value.MetadataSize); - value.UnmarkExtraMetadata(); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - return true; - } + // GetRMWModifiedFieldLength saw PERSIST + _ = logRecord.RemoveExpiration(); } - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - return true; + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); + shouldUpdateEtag = false; + break; case RespCommand.APPEND: // If nothing to append, can avoid copy update. - var appendSize = input.parseState.GetArgSliceByRef(0).Length; - - if (appendSize == 0) + var appendValue = input.parseState.GetArgSliceByRef(0); + var appendLength = appendValue.Length; + if (appendLength > 0) { - CopyValueLengthToOutput(ref value, ref output, functionsState.etagState.etagSkippedStart); - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); - return true; + // Try to grow in place. + var originalLength = logRecord.ValueSpan.Length; + if (!logRecord.TrySetValueLength(originalLength + appendLength, ref sizeInfo)) + return false; + + // Append the new value with the client input at the end of the old data + appendValue. + // Append the new value with the client input at the end of the old data + ReadOnlySpan.CopyTo(logRecord.ValueSpan.Slice(originalLength)); + if (!CopyValueLengthToOutput(logRecord.ValueSpan, ref output)) + return false; + break; } - return false; + // reset etag state that may have been initialized earlier, but don't update etag + ETagState.ResetState(ref functionsState.etagState); + return CopyValueLengthToOutput(logRecord.ValueSpan, ref output); + default: if (cmd > RespCommandExtensions.LastValidCommand) { if (shouldUpdateEtag) { - CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); - // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + functionsState.CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); + // reset etag state that may have been initialized earlier but don't update ETag + ETagState.ResetState(ref functionsState.etagState); return true; } @@ -847,42 +873,31 @@ private bool InPlaceUpdaterWorker(ref SpanByte key, ref RawStringInput input, re var expiration = input.arg1; if (expiration == -1) { - // there is existing metadata, but we want to clear it. - // we remove metadata, shift the value, shrink length - if (value.ExtraMetadata > 0) - { - var oldValue = value.AsReadOnlySpan(); - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.UnmarkExtraMetadata(); - oldValue.CopyTo(value.AsSpan()); - value.ShrinkSerializedLength(oldValue.Length); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - } + // There is existing expiration and we want to clear it. + _ = logRecord.RemoveExpiration(); } else if (expiration > 0) { - // there is no existing metadata, but we want to add it. we cannot do in place update. - if (value.ExtraMetadata == 0) return false; - // set expiration to the specific value - value.ExtraMetadata = expiration; + // There is no existing metadata, but we want to add it. Try to do in place update. + if (!logRecord.TrySetExpiration(expiration)) + return false; } + shouldCheckExpiration = false; - var valueLength = value.LengthWithoutMetadata; + var valueLength = logRecord.ValueSpan.Length; (IMemoryOwner Memory, int Length) outp = (output.Memory, 0); - var ret = functions.InPlaceUpdater(key.AsReadOnlySpan(), ref input, value.AsSpan(), ref valueLength, ref outp, ref rmwInfo); - Debug.Assert(valueLength <= value.LengthWithoutMetadata); + var ret = functions.InPlaceUpdater(logRecord.Key, ref input, logRecord.ValueSpan, ref valueLength, ref outp, ref rmwInfo); + Debug.Assert(valueLength <= logRecord.ValueSpan.Length); // Adjust value length if user shrinks it - if (valueLength < value.LengthWithoutMetadata) - { - rmwInfo.ClearExtraValueLength(ref recordInfo, ref value, value.TotalSize); - value.ShrinkSerializedLength(valueLength + value.MetadataSize); - rmwInfo.SetUsedValueLength(ref recordInfo, ref value, value.TotalSize); - } + if (valueLength < logRecord.ValueSpan.Length) + _ = logRecord.TrySetValueLength(valueLength, ref sizeInfo); output.Memory = outp.Memory; output.Length = outp.Length; - return ret; + if (!ret) + return false; + break; } throw new GarnetException("Unsupported operation on input"); } @@ -890,33 +905,32 @@ private bool InPlaceUpdaterWorker(ref SpanByte key, ref RawStringInput input, re // increment the Etag transparently if in place update happened if (shouldUpdateEtag) { - value.SetEtagInPayload(this.functionsState.etagState.etag + 1); + logRecord.TrySetETag(this.functionsState.etagState.ETag + 1); + ETagState.ResetState(ref functionsState.etagState); } - - if (hadRecordPreMutation) + else if (hadETagPreMutation) { // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); } + sizeInfo.AssertOptionals(logRecord.Info, checkExpiration: shouldCheckExpiration); return true; } // NOTE: In the below control flow if you decide to add a new command or modify a command such that it will now do an early return with FALSE, you must make sure you must reset etagState in FunctionState /// - public bool NeedCopyUpdate(ref SpanByte key, ref RawStringInput input, ref SpanByte oldValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public readonly bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { switch (input.header.cmd) { case RespCommand.DELIFGREATER: - if (rmwInfo.RecordInfo.ETag) - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref oldValue); - + if (srcLogRecord.Info.HasETag) + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref srcLogRecord); long etagFromClient = input.parseState.GetLong(0); if (etagFromClient > functionsState.etagState.etag) - { rmwInfo.Action = RMWAction.ExpireAndStop; - } EtagState.ResetState(ref functionsState.etagState); // We always return false because we would rather not create a new record in hybrid log if we don't need to delete the object. @@ -926,24 +940,22 @@ public bool NeedCopyUpdate(ref SpanByte key, ref RawStringInput input, ref SpanB case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - if (rmwInfo.RecordInfo.ETag) - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref oldValue); + if (srcLogRecord.Info.HasETag) + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref srcLogRecord); long etagToCheckWith = input.parseState.GetLong(1); // in IFMATCH we check for equality, in IFGREATER we are checking for sent etag being strictly greater - int comparisonResult = etagToCheckWith.CompareTo(functionsState.etagState.etag); + int comparisonResult = etagToCheckWith.CompareTo(functionsState.etagState.ETag); int expectedResult = input.header.cmd is RespCommand.SETIFMATCH ? 0 : 1; if (comparisonResult == expectedResult) - { return true; - } if (input.header.CheckSetGetFlag()) { // Copy value to output for the GET part of the command. - CopyRespWithEtagData(ref oldValue, ref output, hasEtagInVal: rmwInfo.RecordInfo.ETag, functionsState.etagState.etagSkippedStart, functionsState.memoryPool); + CopyRespWithEtagData(srcLogRecord.ValueSpan, ref output, srcLogRecord.Info.HasETag, functionsState.memoryPool); } else { @@ -951,26 +963,26 @@ public bool NeedCopyUpdate(ref SpanByte key, ref RawStringInput input, ref SpanB var nilResponse = functionsState.nilResp; // *2\r\n: + + \r\n + WriteValAndEtagToDst( - 4 + 1 + NumUtils.CountDigits(functionsState.etagState.etag) + 2 + nilResponse.Length, - ref nilResponse, - functionsState.etagState.etag, + 4 + 1 + NumUtils.CountDigits(functionsState.etagState.ETag) + 2 + nilResponse.Length, + nilResponse, + functionsState.etagState.ETag, ref output, functionsState.memoryPool, writeDirect: true ); } - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; case RespCommand.SETEXNX: // Expired data, return false immediately // ExpireAndResume ensures that we set as new value, since it does not exist - if (oldValue.MetadataSize > 0 && input.header.CheckExpiry(oldValue.ExtraMetadata)) + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) { rmwInfo.Action = RMWAction.ExpireAndResume; - rmwInfo.RecordInfo.ClearHasETag(); + // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; } @@ -979,42 +991,42 @@ public bool NeedCopyUpdate(ref SpanByte key, ref RawStringInput input, ref SpanB if (input.header.CheckSetGetFlag()) { // Copy value to output for the GET part of the command. - CopyRespTo(ref oldValue, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(srcLogRecord.ValueSpan, ref output); } - else if (input.header.CheckWithEtagFlag()) + else if (input.header.CheckWithETagFlag()) { // EXX when unsuccesful will write back NIL - CopyDefaultResp(functionsState.nilResp, ref output); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); } // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; case RespCommand.SETEXXX: // Expired data, return false immediately so we do not set, since it does not exist // ExpireAndStop ensures that caller sees a NOTFOUND status - if (oldValue.MetadataSize > 0 && input.header.CheckExpiry(oldValue.ExtraMetadata)) + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) { - rmwInfo.RecordInfo.ClearHasETag(); rmwInfo.Action = RMWAction.ExpireAndStop; // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; } return true; default: if (input.header.cmd > RespCommandExtensions.LastValidCommand) { - if (rmwInfo.RecordInfo.ETag) + if (srcLogRecord.Info.HasETag) { - CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; } (IMemoryOwner Memory, int Length) outp = (output.Memory, 0); + var ret = functionsState.GetCustomCommandFunctions((ushort)input.header.cmd) - .NeedCopyUpdate(key.AsReadOnlySpan(), ref input, oldValue.AsReadOnlySpan(functionsState.etagState.etagSkippedStart), ref outp); + .NeedCopyUpdate(srcLogRecord.Key, ref input, srcLogRecord.ValueSpan, ref outp); output.Memory = outp.Memory; output.Length = outp.Length; return ret; @@ -1025,28 +1037,30 @@ public bool NeedCopyUpdate(ref SpanByte key, ref RawStringInput input, ref SpanB // NOTE: Before doing any return from this method, please make sure you are calling reset on etagState in functionsState. /// - public bool CopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public readonly bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { // Expired data - if (oldValue.MetadataSize > 0 && input.header.CheckExpiry(oldValue.ExtraMetadata)) + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) { - recordInfo.ClearHasETag(); + _ = dstLogRecord.RemoveETag(); rmwInfo.Action = RMWAction.ExpireAndResume; // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; } - rmwInfo.ClearExtraValueLength(ref recordInfo, ref newValue, newValue.TotalSize); + var oldValue = srcLogRecord.ValueSpan; // reduce redundant length calcs + // Do not pre-get newValue = dstLogRecord.ValueSpan here, because it may change, e.g. moving between inline and overflow RespCommand cmd = input.header.cmd; - bool recordHadEtagPreMutation = recordInfo.ETag; + bool recordHadEtagPreMutation = srcLogRecord.Info.HasETag; bool 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, ref oldValue); + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref srcLogRecord); } switch (cmd) @@ -1055,144 +1069,101 @@ public bool CopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte case RespCommand.SETIFMATCH: // By now the comparison for etag against existing etag has already been done in NeedCopyUpdate shouldUpdateEtag = true; - // Copy input to value - ReadOnlySpan src = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - - // retain metadata unless metadata sent - int metadataSize = input.arg1 != 0 ? sizeof(long) : oldValue.MetadataSize; - Debug.Assert(src.Length + EtagConstants.EtagSize + metadataSize == newValue.Length); - - newValue.ExtraMetadata = oldValue.ExtraMetadata; - if (input.arg1 != 0) - { - newValue.ExtraMetadata = input.arg1; - } - - Span dest = newValue.AsSpan(EtagConstants.EtagSize); - src.CopyTo(dest); - - long etagFromClient = input.parseState.GetLong(1); - - functionsState.etagState.etag = etagFromClient; + var inputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; + if (!dstLogRecord.TrySetValueSpan(inputValue, ref sizeInfo)) + return false; - long etagForResponse = cmd == RespCommand.SETIFMATCH ? functionsState.etagState.etag + 1 : functionsState.etagState.etag; + // change the current etag to the the etag sent from client since rest remains same + functionsState.etagState.ETag = input.parseState.GetLong(1); + if (!dstLogRecord.TrySetETag(functionsState.etagState.ETag + (cmd == RespCommand.SETIFMATCH ? 1 : 0))) + return false; - recordInfo.SetHasETag(); + if (sizeInfo.FieldInfo.HasExpiration && !dstLogRecord.TrySetExpiration(input.arg1 != 0 ? input.arg1 : srcLogRecord.Expiration)) + return false; - // Write Etag and Val back to Client - // write back array of the format [etag, nil] - var nilResp = functionsState.nilResp; + // Write Etag and Val back to Client as an array of the format [etag, nil] + long eTagForResponse = cmd == RespCommand.SETIFMATCH ? functionsState.etagState.ETag + 1 : functionsState.etagState.ETag; // *2\r\n: + + \r\n + - var numDigitsInEtag = NumUtils.CountDigits(etagForResponse); - WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + nilResp.Length, ref nilResp, etagForResponse, ref output, functionsState.memoryPool, writeDirect: true); + var numDigitsInEtag = NumUtils.CountDigits(eTagForResponse); + WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + functionsState.nilResp.Length, functionsState.nilResp, eTagForResponse, ref output, functionsState.memoryPool, writeDirect: true); + shouldUpdateEtag = false; // since we already updated the ETag break; case RespCommand.SET: case RespCommand.SETEXXX: - var nextUpdateEtagOffset = functionsState.etagState.etagSkippedStart; - var nextUpdateEtagAccountedLength = functionsState.etagState.etagAccountedLength; - bool inputWithEtag = input.header.CheckWithEtagFlag(); + bool inputHeaderHasEtag = input.header.CheckWithETagFlag(); - // only when both are not false && false or true and true, do we need to readjust - if (inputWithEtag != shouldUpdateEtag) - { - // in the common path the above condition is skipped - if (inputWithEtag) - { - // nextUpdate will add etag but currently there is no etag - nextUpdateEtagOffset = EtagConstants.EtagSize; - shouldUpdateEtag = true; - recordInfo.SetHasETag(); - } - else - { - // nextUpdate will remove etag but currentyly there is an etag - nextUpdateEtagOffset = 0; - shouldUpdateEtag = false; - recordInfo.ClearHasETag(); - } - } + if (inputHeaderHasEtag != shouldUpdateEtag) + shouldUpdateEtag = inputHeaderHasEtag; // Check if SetGet flag is set if (input.header.CheckSetGetFlag()) { - Debug.Assert(!input.header.CheckWithEtagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); + Debug.Assert(!input.header.CheckWithETagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); // Copy value to output for the GET part of the command. - CopyRespTo(ref oldValue, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(oldValue, ref output); } - // Copy input to value var newInputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - metadataSize = input.arg1 == 0 ? 0 : sizeof(long); + Debug.Assert(newInputValue.Length == dstLogRecord.ValueSpan.Length); - // new value when allocated should have 8 bytes more if the previous record had etag and the cmd was not SETEXXX - Debug.Assert(newInputValue.Length + metadataSize + nextUpdateEtagOffset == newValue.Length); - - newValue.ExtraMetadata = input.arg1; - newInputValue.CopyTo(newValue.AsSpan(nextUpdateEtagOffset)); + // Copy input to value, along with optionals from source record including Expiration. + if (!dstLogRecord.TrySetValueSpan(newInputValue, ref sizeInfo) || !dstLogRecord.TryCopyOptionals(ref srcLogRecord, ref sizeInfo)) + return false; - if (inputWithEtag) + if (inputHeaderHasEtag != shouldUpdateEtag) + shouldUpdateEtag = inputHeaderHasEtag; + if (inputHeaderHasEtag) { - CopyRespNumber(functionsState.etagState.etag + 1, ref output); + var newETag = functionsState.etagState.ETag + 1; + dstLogRecord.TrySetETag(newETag); + functionsState.CopyRespNumber(newETag, ref output); } + else + dstLogRecord.RemoveETag(); + 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: case RespCommand.SETKEEPTTL: - nextUpdateEtagOffset = functionsState.etagState.etagSkippedStart; - nextUpdateEtagAccountedLength = functionsState.etagState.etagAccountedLength; - inputWithEtag = input.header.CheckWithEtagFlag(); - - // only when both are not false && false or true and true, do we need to readjust - if (inputWithEtag != shouldUpdateEtag) - { - // in the common path the above condition is skipped - if (inputWithEtag) - { - // nextUpdate will add etag but currently there is no etag - nextUpdateEtagOffset = EtagConstants.EtagSize; - shouldUpdateEtag = true; - recordInfo.SetHasETag(); - } - else - { - shouldUpdateEtag = false; - // nextUpdate will remove etag but currentyly there is an etag - nextUpdateEtagOffset = 0; - recordInfo.ClearHasETag(); - } - } - - var setValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - - Debug.Assert(oldValue.MetadataSize + setValue.Length + nextUpdateEtagOffset == newValue.Length); + // If the user calls withetag then we need to either update an existing etag and set the value + // or set the value with an initial etag and increment it. If withEtag is called we return the etag back to the user + inputHeaderHasEtag = input.header.CheckWithETagFlag(); - // Check if SetGet flag is set + // If the SetGet flag is set, copy the current value to output for the GET part of the command. if (input.header.CheckSetGetFlag()) { - Debug.Assert(!input.header.CheckWithEtagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); + Debug.Assert(!input.header.CheckWithETagFlag(), "SET GET CANNNOT BE CALLED WITH WITHETAG"); + // Copy value to output for the GET part of the command. - CopyRespTo(ref oldValue, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(srcLogRecord.ValueSpan, ref output); } - // Copy input to value, retain metadata of oldValue - newValue.ExtraMetadata = oldValue.ExtraMetadata; - setValue.CopyTo(newValue.AsSpan(nextUpdateEtagOffset)); + inputValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; + if (!dstLogRecord.TrySetValueSpan(inputValue, ref sizeInfo)) + return false; - if (inputWithEtag) + if (inputHeaderHasEtag != shouldUpdateEtag) + shouldUpdateEtag = inputHeaderHasEtag; + if (inputHeaderHasEtag) { - CopyRespNumber(functionsState.etagState.etag + 1, ref output); + var newETag = functionsState.etagState.ETag + 1; + dstLogRecord.TrySetETag(newETag); + functionsState.CopyRespNumber(newETag, ref output); } + else + dstLogRecord.RemoveETag(); + shouldUpdateEtag = false; // since we already updated the ETag break; case RespCommand.EXPIRE: case RespCommand.PEXPIRE: shouldUpdateEtag = false; - - var expiryExists = oldValue.MetadataSize > 0; - var expiryValue = input.parseState.GetLong(0); var tsExpiry = input.header.cmd == RespCommand.EXPIRE ? TimeSpan.FromSeconds(expiryValue) @@ -1200,51 +1171,62 @@ public bool CopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte var expiryTicks = DateTimeOffset.UtcNow.Ticks + tsExpiry.Ticks; var expireOption = (ExpireOption)input.arg1; - EvaluateExpireCopyUpdate(expireOption, expiryExists, expiryTicks, ref oldValue, ref newValue, ref output); + // First copy the old Value and non-Expiration optionals to the new record. This will also ensure space for expiration. + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + + if (!EvaluateExpireCopyUpdate(ref dstLogRecord, ref sizeInfo, expireOption, expiryTicks, dstLogRecord.ValueSpan, ref output)) + return false; break; case RespCommand.PEXPIREAT: case RespCommand.EXPIREAT: - expiryExists = oldValue.MetadataSize > 0; shouldUpdateEtag = false; - var expiryTimestamp = input.parseState.GetLong(0); expiryTicks = input.header.cmd == RespCommand.PEXPIREAT ? ConvertUtils.UnixTimestampInMillisecondsToTicks(expiryTimestamp) : ConvertUtils.UnixTimestampInSecondsToTicks(expiryTimestamp); expireOption = (ExpireOption)input.arg1; - EvaluateExpireCopyUpdate(expireOption, expiryExists, expiryTicks, ref oldValue, ref newValue, ref output); + // First copy the old Value and non-Expiration optionals to the new record. This will also ensure space for expiration. + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + + if (!EvaluateExpireCopyUpdate(ref dstLogRecord, ref sizeInfo, expireOption, expiryTicks, dstLogRecord.ValueSpan, ref output)) + return false; break; case RespCommand.PERSIST: shouldUpdateEtag = false; - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - if (oldValue.MetadataSize != 0) + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + if (srcLogRecord.Info.HasExpiration) { - newValue.AsSpan().CopyTo(newValue.AsSpanWithMetadata()); - newValue.ShrinkSerializedLength(newValue.Length - newValue.MetadataSize); - newValue.UnmarkExtraMetadata(); - output.SpanByte.AsSpan()[0] = 1; + dstLogRecord.RemoveExpiration(); + output.SpanByte.Span[0] = 1; } break; case RespCommand.INCR: - TryCopyUpdateNumber(ref oldValue, ref newValue, ref output, input: 1, functionsState.etagState.etagSkippedStart); + if (!TryCopyUpdateNumber(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref output, input: 1)) + return false; break; case RespCommand.DECR: - TryCopyUpdateNumber(ref oldValue, ref newValue, ref output, input: -1, functionsState.etagState.etagSkippedStart); + if (!TryCopyUpdateNumber(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref output, input: -1)) + return false; break; case RespCommand.INCRBY: var incrBy = input.arg1; - TryCopyUpdateNumber(ref oldValue, ref newValue, ref output, input: incrBy, functionsState.etagState.etagSkippedStart); + if (!TryCopyUpdateNumber(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref output, input: incrBy)) + return false; break; case RespCommand.DECRBY: var decrBy = input.arg1; - TryCopyUpdateNumber(ref oldValue, ref newValue, ref output, input: -decrBy, functionsState.etagState.etagSkippedStart); + if (!TryCopyUpdateNumber(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref output, input: -decrBy)) + return false; break; case RespCommand.INCRBYFLOAT: @@ -1252,160 +1234,306 @@ public bool CopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte if (!input.parseState.TryGetDouble(0, out var incrByFloat)) { // Move to tail of the log - oldValue.CopyTo(ref newValue); + oldValue.CopyTo(dstLogRecord.ValueSpan); break; } - TryCopyUpdateNumber(ref oldValue, ref newValue, ref output, input: incrByFloat, functionsState.etagState.etagSkippedStart); + _ = TryCopyUpdateNumber(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref output, input: incrByFloat); break; case RespCommand.SETBIT: var bOffset = input.arg1; var bSetVal = (byte)(input.parseState.GetArgSliceByRef(1).ReadOnlySpan[0] - '0'); - Buffer.MemoryCopy(oldValue.ToPointer(), newValue.ToPointer(), newValue.Length, oldValue.Length); - var oldValSet = BitmapManager.UpdateBitmap(newValue.ToPointer(), bOffset, bSetVal); + + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + + // Some duplicate code to avoid "fixed" when possible + var newValue = dstLogRecord.ValueSpan; + byte* oldValuePtr; + byte oldValSet; + if (srcLogRecord.IsPinnedValue) + { + oldValuePtr = srcLogRecord.PinnedValuePointer; + if (dstLogRecord.IsPinnedValue) + { + var newValuePtr = dstLogRecord.PinnedValuePointer; + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + oldValSet = BitmapManager.UpdateBitmap(newValuePtr, bOffset, bSetVal); + } + else + { + fixed (byte* newValuePtr = dstLogRecord.ValueSpan) + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + oldValSet = BitmapManager.UpdateBitmap(newValuePtr, bOffset, bSetVal); + } + } + } + else + { + fixed (byte* oldPtr = srcLogRecord.ValueSpan) + { + oldValuePtr = oldPtr; + if (dstLogRecord.IsPinnedValue) + { + var newValuePtr = dstLogRecord.PinnedValuePointer; + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + oldValSet = BitmapManager.UpdateBitmap(newValuePtr, bOffset, bSetVal); + } + else + { + fixed (byte* newValuePtr = dstLogRecord.ValueSpan) + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + oldValSet = BitmapManager.UpdateBitmap(newValuePtr, bOffset, bSetVal); + } + } + } + } + if (oldValSet == 0) - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output); else - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output); break; case RespCommand.BITFIELD: var bitFieldArgs = GetBitFieldArguments(ref input); - Buffer.MemoryCopy(oldValue.ToPointer() + functionsState.etagState.etagSkippedStart, newValue.ToPointer() + functionsState.etagState.etagSkippedStart, newValue.Length - functionsState.etagState.etagSkippedStart, oldValue.Length - functionsState.etagState.etagSkippedStart); - var (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, - newValue.ToPointer() + functionsState.etagState.etagSkippedStart, - newValue.Length - functionsState.etagState.etagSkippedStart); + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + + newValue = dstLogRecord.ValueSpan; + long bitfieldReturnValue; + bool overflow; + if (dstLogRecord.IsPinnedValue) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, dstLogRecord.PinnedValuePointer, newValue.Length); + else + fixed(byte* newValuePtr = newValue) + (bitfieldReturnValue, overflow) = BitmapManager.BitFieldExecute(bitFieldArgs, newValuePtr, newValue.Length); if (!overflow) - CopyRespNumber(bitfieldReturnValue, ref output); + functionsState.CopyRespNumber(bitfieldReturnValue, ref output); else - CopyDefaultResp(functionsState.nilResp, ref output); + functionsState.CopyDefaultResp(functionsState.nilResp, ref output); break; case RespCommand.BITFIELD_RO: var bitFieldArgs_RO = GetBitFieldArguments(ref input); - Buffer.MemoryCopy(oldValue.ToPointer() + functionsState.etagState.etagSkippedStart, newValue.ToPointer() + functionsState.etagState.etagSkippedStart, newValue.Length - functionsState.etagState.etagSkippedStart, oldValue.Length - functionsState.etagState.etagSkippedStart); - var bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, - newValue.ToPointer() + functionsState.etagState.etagSkippedStart, - newValue.Length - functionsState.etagState.etagSkippedStart - ); - CopyRespNumber(bitfieldReturnValue_RO, ref output); + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + + newValue = dstLogRecord.ValueSpan; + long bitfieldReturnValue_RO; + if (dstLogRecord.IsPinnedValue) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, dstLogRecord.PinnedValuePointer, newValue.Length); + else + fixed(byte* newValuePtr = newValue) + bitfieldReturnValue_RO = BitmapManager.BitFieldExecute_RO(bitFieldArgs_RO, newValuePtr, newValue.Length); + + functionsState.CopyRespNumber(bitfieldReturnValue_RO, ref output); break; case RespCommand.PFADD: var updated = false; - var newValPtr = newValue.ToPointer(); - var oldValPtr = oldValue.ToPointer(); + newValue = dstLogRecord.ValueSpan; - if (newValue.Length != oldValue.Length) - updated = HyperLogLog.DefaultHLL.CopyUpdate(ref input, oldValPtr, newValPtr, newValue.Length); + if (!dstLogRecord.TryCopyOptionals(ref srcLogRecord, ref sizeInfo)) + return false; + + // Some duplicate code to avoid "fixed" when possible + newValue = dstLogRecord.ValueSpan; + if (srcLogRecord.IsPinnedValue) + { + oldValuePtr = srcLogRecord.PinnedValuePointer; + if (dstLogRecord.IsPinnedValue) + { + var newValuePtr = dstLogRecord.PinnedValuePointer; + if (newValue.Length != oldValue.Length) + updated = HyperLogLog.DefaultHLL.CopyUpdate(ref input, oldValuePtr, newValuePtr, newValue.Length); + else + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + _ = HyperLogLog.DefaultHLL.Update(ref input, newValuePtr, newValue.Length, ref updated); + } + } + else + { + fixed (byte* newValuePtr = dstLogRecord.ValueSpan) + { + if (newValue.Length != oldValue.Length) + updated = HyperLogLog.DefaultHLL.CopyUpdate(ref input, oldValuePtr, newValuePtr, newValue.Length); + else + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + _ = HyperLogLog.DefaultHLL.Update(ref input, newValuePtr, newValue.Length, ref updated); + } + } + } + } else { - Buffer.MemoryCopy(oldValPtr, newValPtr, newValue.Length, oldValue.Length); - HyperLogLog.DefaultHLL.Update(ref input, newValPtr, newValue.Length, ref updated); + fixed (byte* oldPtr = srcLogRecord.ValueSpan) + { + oldValuePtr = oldPtr; + if (dstLogRecord.IsPinnedValue) + { + var newValuePtr = dstLogRecord.PinnedValuePointer; + if (newValue.Length != oldValue.Length) + updated = HyperLogLog.DefaultHLL.CopyUpdate(ref input, oldValuePtr, newValuePtr, newValue.Length); + else + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + _ = HyperLogLog.DefaultHLL.Update(ref input, newValuePtr, newValue.Length, ref updated); + } + } + else + { + fixed (byte* newValuePtr = dstLogRecord.ValueSpan) + { + if (newValue.Length != oldValue.Length) + updated = HyperLogLog.DefaultHLL.CopyUpdate(ref input, oldValuePtr, newValuePtr, newValue.Length); + else + { + Buffer.MemoryCopy(oldValuePtr, newValuePtr, newValue.Length, oldValue.Length); + _ = HyperLogLog.DefaultHLL.Update(ref input, newValuePtr, newValue.Length, ref updated); + } + } + } + } } + *output.SpanByte.ToPointer() = updated ? (byte)1 : (byte)0; break; case RespCommand.PFMERGE: + if (!dstLogRecord.TryCopyOptionals(ref srcLogRecord, ref sizeInfo)) + return false; + + // Explanation of variables: //srcA offset: [hll allocated size = 4 byte] + [hll data structure] //memcpy +4 (skip len size) - var srcHLLPtr = input.parseState.GetArgSliceByRef(0).SpanByte.ToPointer(); // HLL merging from - var oldDstHLLPtr = oldValue.ToPointer(); // original HLL merging to (too small to hold its data plus srcA) - var newDstHLLPtr = newValue.ToPointer(); // new HLL merging to (large enough to hold srcA and srcB + var srcHLLPtr = input.parseState.GetArgSliceByRef(0).ToPointer(); // HLL merging from + // byte* oldDstHLLPtr = oldValue.ToPointer(); // original HLL merging to (too small to hold its data plus srcA) + // byte* newDstHLLPtr = newValue.ToPointer(); // new HLL merging to (large enough to hold srcA and srcB + + // Some duplicate code to avoid "fixed" when possible + newValue = dstLogRecord.ValueSpan; + if (srcLogRecord.IsPinnedValue) + { + var oldDstHLLPtr = srcLogRecord.PinnedValuePointer; + if (dstLogRecord.IsPinnedValue) + { + var newDstHLLPtr = dstLogRecord.PinnedValuePointer; + HyperLogLog.DefaultHLL.CopyUpdateMerge(srcHLLPtr, oldDstHLLPtr, newDstHLLPtr, oldValue.Length, newValue.Length); + } + else + { + fixed (byte* newDstHLLPtr = dstLogRecord.ValueSpan) + HyperLogLog.DefaultHLL.CopyUpdateMerge(srcHLLPtr, oldDstHLLPtr, newDstHLLPtr, oldValue.Length, newValue.Length); + } + } + else + { + fixed (byte* oldDstHLLPtr = srcLogRecord.ValueSpan) + { + if (dstLogRecord.IsPinnedValue) + { + var newDstHLLPtr = dstLogRecord.PinnedValuePointer; + HyperLogLog.DefaultHLL.CopyUpdateMerge(srcHLLPtr, oldDstHLLPtr, newDstHLLPtr, oldValue.Length, newValue.Length); + } + else + { + fixed (byte* newDstHLLPtr = dstLogRecord.ValueSpan) + HyperLogLog.DefaultHLL.CopyUpdateMerge(srcHLLPtr, oldDstHLLPtr, newDstHLLPtr, oldValue.Length, newValue.Length); + } + } + } - HyperLogLog.DefaultHLL.CopyUpdateMerge(srcHLLPtr, oldDstHLLPtr, newDstHLLPtr, oldValue.Length, newValue.Length); break; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); - oldValue.CopyTo(ref newValue); - newInputValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - newInputValue.CopyTo(newValue.AsSpan(functionsState.etagState.etagSkippedStart).Slice(offset)); + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; - CopyValueLengthToOutput(ref newValue, ref output, functionsState.etagState.etagSkippedStart); + newValue = dstLogRecord.ValueSpan; + input.parseState.GetArgSliceByRef(1).ReadOnlySpan.CopyTo(newValue.Slice(offset)); + + _ = CopyValueLengthToOutput(newValue, ref output); break; case RespCommand.GETDEL: // Copy value to output for the GET part of the command. // Then, set ExpireAndStop action to delete the record. - CopyRespTo(ref oldValue, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(oldValue, ref output); rmwInfo.Action = RMWAction.ExpireAndStop; // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return false; case RespCommand.GETEX: shouldUpdateEtag = false; - CopyRespTo(ref oldValue, ref output, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(oldValue, ref output); + + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + newValue = dstLogRecord.ValueSpan; + Debug.Assert(newValue.Length == oldValue.Length); if (input.arg1 > 0) { - Debug.Assert(newValue.Length == oldValue.Length + sizeof(long)); - byte* pbOutput = stackalloc byte[ObjectOutputHeader.Size]; - var _output = new SpanByteAndMemory(SpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); + var pbOutput = stackalloc byte[ObjectOutputHeader.Size]; + var _output = new SpanByteAndMemory(PinnedSpanByte.FromPinnedPointer(pbOutput, ObjectOutputHeader.Size)); var newExpiry = input.arg1; - EvaluateExpireCopyUpdate(ExpireOption.None, expiryExists: oldValue.MetadataSize > 0, newExpiry, ref oldValue, ref newValue, ref _output); + if (!EvaluateExpireCopyUpdate(ref dstLogRecord, ref sizeInfo, ExpireOption.None, newExpiry, newValue, ref _output)) + return false; } - oldValue.AsReadOnlySpan().CopyTo(newValue.AsSpan()); - if (input.parseState.Count > 0) { - var persist = input.parseState.GetArgSliceByRef(0).ReadOnlySpan - .EqualsUpperCaseSpanIgnoringCase(CmdStrings.PERSIST); - + var persist = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase(CmdStrings.PERSIST); if (persist) // Persist the key - { - newValue.AsSpan().CopyTo(newValue.AsSpanWithMetadata()); - newValue.ShrinkSerializedLength(newValue.Length - newValue.MetadataSize); - newValue.UnmarkExtraMetadata(); - } + _ = dstLogRecord.RemoveExpiration(); } break; case RespCommand.APPEND: - // Copy any existing value with metadata to thew new value - oldValue.CopyTo(ref newValue); - var appendValue = input.parseState.GetArgSliceByRef(0); + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; // Append the new value with the client input at the end of the old data - appendValue.ReadOnlySpan.CopyTo(newValue.AsSpan().Slice(oldValue.LengthWithoutMetadata)); + newValue = dstLogRecord.ValueSpan; + appendValue.ReadOnlySpan.CopyTo(newValue.Slice(oldValue.Length)); - CopyValueLengthToOutput(ref newValue, ref output, functionsState.etagState.etagSkippedStart); + _ = CopyValueLengthToOutput(newValue, ref output); break; default: if (input.header.cmd > RespCommandExtensions.LastValidCommand) { - if (recordInfo.ETag) + if (srcLogRecord.Info.HasETag) { - CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); + functionsState.CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); // reset etag state that may have been initialized earlier - EtagState.ResetState(ref functionsState.etagState); + ETagState.ResetState(ref functionsState.etagState); return true; } var functions = functionsState.GetCustomCommandFunctions((ushort)input.header.cmd); var expiration = input.arg1; - if (expiration == 0) - { - // We want to retain the old metadata - newValue.ExtraMetadata = oldValue.ExtraMetadata; - } - else if (expiration > 0) + if (expiration > 0) { - // We want to add the given expiration - newValue.ExtraMetadata = expiration; + // We want to update to the given expiration + if (!dstLogRecord.TrySetExpiration(expiration)) + return false; } (IMemoryOwner Memory, int Length) outp = (output.Memory, 0); - var ret = functions - .CopyUpdater(key.AsReadOnlySpan(), ref input, oldValue.AsReadOnlySpan(functionsState.etagState.etagSkippedStart), newValue.AsSpan(functionsState.etagState.etagSkippedStart), ref outp, ref rmwInfo); + var ret = functions.CopyUpdater(dstLogRecord.Key, ref input, oldValue, dstLogRecord.ValueSpan, ref outp, ref rmwInfo); output.Memory = outp.Memory; output.Length = outp.Length; return ret; @@ -1413,28 +1541,29 @@ public bool CopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte throw new GarnetException("Unsupported operation on input"); } - rmwInfo.SetUsedValueLength(ref recordInfo, ref newValue, newValue.TotalSize); if (shouldUpdateEtag) { - if (cmd is not RespCommand.SETIFGREATER) - functionsState.etagState.etag++; - - newValue.SetEtagInPayload(functionsState.etagState.etag); - EtagState.ResetState(ref functionsState.etagState); + dstLogRecord.TrySetETag(functionsState.etagState.ETag + 1); + ETagState.ResetState(ref functionsState.etagState); } else if (recordHadEtagPreMutation) - EtagState.ResetState(ref functionsState.etagState); + { + // reset etag state that may have been initialized earlier + ETagState.ResetState(ref functionsState.etagState); + } + sizeInfo.AssertOptionals(dstLogRecord.Info); return true; } /// - public bool PostCopyUpdater(ref SpanByte key, ref RawStringInput input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public readonly bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogRMW(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + WriteLogRMW(dstLogRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); return true; } } diff --git a/libs/server/Storage/Functions/MainStore/ReadMethods.cs b/libs/server/Storage/Functions/MainStore/ReadMethods.cs index fc686c63bf7..2648182dd22 100644 --- a/libs/server/Storage/Functions/MainStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/MainStore/ReadMethods.cs @@ -11,143 +11,68 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// - public bool SingleReader( - ref SpanByte key, ref RawStringInput input, - ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) + public bool Reader(ref TSourceLogRecord srcLogRecord, ref RawStringInput input, ref SpanByteAndMemory output, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord { - if (value.MetadataSize != 0 && CheckExpiry(ref value)) - { - readInfo.RecordInfo.ClearHasETag(); + if (CheckExpiry(ref srcLogRecord)) return false; - } var cmd = input.header.cmd; - + var value = srcLogRecord.ValueSpan; // reduce redundant length calculations if (cmd == RespCommand.GETIFNOTMATCH) { - if (handleGetIfNotMatch(ref input, ref value, ref dst, ref readInfo)) + if (handleGetIfNotMatch(ref srcLogRecord, ref input, ref output, ref readInfo)) return true; } else if (cmd > RespCommandExtensions.LastValidCommand) { - if (readInfo.RecordInfo.ETag) + if (srcLogRecord.Info.HasETag) { - CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref dst); + functionsState.CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref output); return true; } - var valueLength = value.LengthWithoutMetadata; - (IMemoryOwner Memory, int Length) output = (dst.Memory, 0); + var valueLength = value.Length; + (IMemoryOwner Memory, int Length) memoryAndLength = (output.Memory, 0); var ret = functionsState.GetCustomCommandFunctions((ushort)cmd) - .Reader(key.AsReadOnlySpan(), ref input, value.AsReadOnlySpan(), ref output, ref readInfo); - Debug.Assert(valueLength <= value.LengthWithoutMetadata); - dst.Memory = output.Memory; - dst.Length = output.Length; + .Reader(srcLogRecord.Key, ref input, value, ref memoryAndLength, ref readInfo); + Debug.Assert(valueLength <= value.Length); + (output.Memory, output.Length) = memoryAndLength; return ret; } - if (readInfo.RecordInfo.ETag) - { - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); - } + if (srcLogRecord.Info.HasETag) + ETagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref srcLogRecord); // Unless the command explicitly asks for the ETag in response, we do not write back the ETag - if (cmd is (RespCommand.GETWITHETAG or RespCommand.GETIFNOTMATCH)) + if (cmd is RespCommand.GETWITHETAG or RespCommand.GETIFNOTMATCH) { - CopyRespWithEtagData(ref value, ref dst, readInfo.RecordInfo.ETag, functionsState.etagState.etagSkippedStart, functionsState.memoryPool); - EtagState.ResetState(ref functionsState.etagState); + CopyRespWithEtagData(value, ref output, srcLogRecord.Info.HasETag, functionsState.memoryPool); + ETagState.ResetState(ref functionsState.etagState); return true; } if (cmd == RespCommand.NONE) - CopyRespTo(ref value, ref dst, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); + CopyRespTo(value, ref output); else - { - CopyRespToWithInput(ref input, ref value, ref dst, readInfo.IsFromPending); - } - - if (readInfo.RecordInfo.ETag) - { - EtagState.ResetState(ref functionsState.etagState); - } - - return true; - } + CopyRespToWithInput(ref srcLogRecord, ref input, ref output, readInfo.IsFromPending); - /// - public bool ConcurrentReader( - ref SpanByte key, ref RawStringInput input, ref SpanByte value, - ref SpanByteAndMemory dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - if (value.MetadataSize != 0 && CheckExpiry(ref value)) - { - recordInfo.ClearHasETag(); - return false; - } - - var cmd = input.header.cmd; - - if (cmd == RespCommand.GETIFNOTMATCH) - { - if (handleGetIfNotMatch(ref input, ref value, ref dst, ref readInfo)) - return true; - } - else if (cmd > RespCommandExtensions.LastValidCommand) - { - if (readInfo.RecordInfo.ETag) - { - CopyDefaultResp(CmdStrings.RESP_ERR_ETAG_ON_CUSTOM_PROC, ref dst); - return true; - } - - var valueLength = value.LengthWithoutMetadata; - (IMemoryOwner Memory, int Length) output = (dst.Memory, 0); - var ret = functionsState.GetCustomCommandFunctions((ushort)cmd) - .Reader(key.AsReadOnlySpan(), ref input, value.AsReadOnlySpan(), ref output, ref readInfo); - Debug.Assert(valueLength <= value.LengthWithoutMetadata); - dst.Memory = output.Memory; - dst.Length = output.Length; - return ret; - } - - if (readInfo.RecordInfo.ETag) - { - EtagState.SetValsForRecordWithEtag(ref functionsState.etagState, ref value); - } - - // Unless the command explicitly asks for the ETag in response, we do not write back the ETag - if (cmd is (RespCommand.GETWITHETAG or RespCommand.GETIFNOTMATCH)) - { - CopyRespWithEtagData(ref value, ref dst, readInfo.RecordInfo.ETag, functionsState.etagState.etagSkippedStart, functionsState.memoryPool); - EtagState.ResetState(ref functionsState.etagState); - return true; - } - - - if (cmd == RespCommand.NONE) - CopyRespTo(ref value, ref dst, functionsState.etagState.etagSkippedStart, functionsState.etagState.etagAccountedLength); - else - { - CopyRespToWithInput(ref input, ref value, ref dst, readInfo.IsFromPending); - } - - if (readInfo.RecordInfo.ETag) - { - EtagState.ResetState(ref functionsState.etagState); - } + if (srcLogRecord.Info.HasETag) + ETagState.ResetState(ref functionsState.etagState); return true; } - private bool handleGetIfNotMatch(ref RawStringInput input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) + private bool handleGetIfNotMatch(ref TSourceLogRecord srcLogRecord, ref RawStringInput input, ref SpanByteAndMemory dst, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord { // Any value without an etag is treated the same as a value with an etag long etagToMatchAgainst = input.parseState.GetLong(0); - long existingEtag = readInfo.RecordInfo.ETag ? value.GetEtagInPayload() : EtagConstants.NoETag; + long existingEtag = srcLogRecord.ETag; if (existingEtag == etagToMatchAgainst) { @@ -155,7 +80,7 @@ private bool handleGetIfNotMatch(ref RawStringInput input, ref SpanByte value, r var nilResp = functionsState.nilResp; // *2\r\n: + + \r\n + var numDigitsInEtag = NumUtils.CountDigits(existingEtag); - WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + nilResp.Length, ref nilResp, existingEtag, ref dst, functionsState.memoryPool, writeDirect: true); + WriteValAndEtagToDst(4 + 1 + numDigitsInEtag + 2 + nilResp.Length, nilResp, existingEtag, ref dst, functionsState.memoryPool, writeDirect: true); return true; } diff --git a/libs/server/Storage/Functions/MainStore/UpsertMethods.cs b/libs/server/Storage/Functions/MainStore/UpsertMethods.cs index 4bf44de3b3c..1f1b074f156 100644 --- a/libs/server/Storage/Functions/MainStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/MainStore/UpsertMethods.cs @@ -1,6 +1,9 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; +using System.Diagnostics; +using Garnet.common; using Tsavorite.core; namespace Garnet.server @@ -8,41 +11,97 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// - public bool SingleWriter(ref SpanByte key, ref RawStringInput input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - // Since upsert may be on existing key we need to wipe out the record info property - recordInfo.ClearHasETag(); - return SpanByteFunctions.DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo, input.arg1); + if (!dstLogRecord.TrySetValueSpan(srcValue, ref sizeInfo)) + return false; + if (input.arg1 != 0 && !dstLogRecord.TrySetExpiration(input.arg1)) + return false; + sizeInfo.AssertOptionals(dstLogRecord.Info); + return true; } /// - public void PostSingleWriter(ref SpanByte key, ref RawStringInput input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason) + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, IHeapObject srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + => throw new GarnetException("String store should not be called with IHeapObject"); + + /// + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref TSourceLogRecord inputLogRecord, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + if (inputLogRecord.Info.ValueIsObject) + throw new GarnetException("String store should not be called with IHeapObject"); + return dstLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + } + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, srcValue, upsertInfo.Version, upsertInfo.SessionID); + } + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, IHeapObject srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + => throw new GarnetException("String store should not be called with IHeapObject"); + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref TSourceLogRecord inputLogRecord, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord { functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); - if (reason == WriteReason.Upsert && functionsState.appendOnlyFile != null) - WriteLogUpsert(ref key, ref input, ref src, upsertInfo.Version, upsertInfo.SessionID); + if (functionsState.appendOnlyFile != null) + { + Debug.Assert(!inputLogRecord.Info.ValueIsObject, "String store should not be called with IHeapObject"); + WriteLogUpsert(logRecord.Key, ref input, inputLogRecord.ValueSpan, upsertInfo.Version, upsertInfo.SessionID); + } } /// - public bool ConcurrentWriter(ref SpanByte key, ref RawStringInput input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - // Since upsert may be on existing key we need to wipe out the record info property - recordInfo.ClearHasETag(); - if (ConcurrentWriterWorker(ref src, ref dst, ref input, ref upsertInfo, ref recordInfo)) + if (!logRecord.TrySetValueSpan(srcValue, ref 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); + } + else + ok = logRecord.RemoveETag(); + } + if (ok) { - if (!upsertInfo.RecordInfo.Modified) + sizeInfo.AssertOptionals(logRecord.Info); + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogUpsert(ref key, ref input, ref src, upsertInfo.Version, upsertInfo.SessionID); + WriteLogUpsert(logRecord.Key, ref input, srcValue, upsertInfo.Version, upsertInfo.SessionID); return true; } return false; } - static bool ConcurrentWriterWorker(ref SpanByte src, ref SpanByte dst, ref RawStringInput input, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - => SpanByteFunctions.DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo, input.arg1); + /// + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, IHeapObject srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + => throw new GarnetException("String store should not be called with IHeapObject"); + + /// + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref RawStringInput input, ref TSourceLogRecord inputLogRecord, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + if (inputLogRecord.Info.ValueIsObject) + throw new GarnetException("String store should not be called with IHeapObject"); + return logRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + } } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs index e8a655f5cd7..87268eceeda 100644 --- a/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/MainStore/VarLenInputMethods.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using Garnet.common; using Tsavorite.core; @@ -9,133 +10,201 @@ namespace Garnet.server /// /// Callback functions for main store /// - public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions + public readonly unsafe partial struct MainSessionFunctions : ISessionFunctions { /// /// Parse ASCII byte array into long and validate that only contains ASCII decimal characters /// - /// Length of byte array - /// Pointer to byte array + /// Source string to evaluate /// Parsed long value /// True if input contained only ASCII decimal characters, otherwise false - static bool IsValidNumber(int length, byte* source, out long val) + static bool IsValidNumber(ReadOnlySpan source, out long val) { - val = 0; try { // Check for valid number - if (!NumUtils.TryReadInt64(length, source, out val)) - { - // Signal value is not a valid number - return false; - } + return NumUtils.TryReadInt64(source, out val); + } + catch + { + // Signal value is not a valid number + val = 0; + return false; + } + } + + /// + /// Parse ASCII byte array into long and validate that only contains ASCII decimal characters + /// + /// Source string to evaluate + /// Parsed long value + /// True if input contained only ASCII decimal characters, otherwise false + static bool IsValidNumber(byte* source, int sourceLen, out long val) + { + try + { + // Check for valid number + return NumUtils.TryReadInt64(sourceLen, source, out val); } catch { // Signal value is not a valid number + val = 0; return false; } - return true; } /// /// Parse ASCII byte array into double and validate that only contains ASCII decimal characters /// - /// Length of byte array - /// Pointer to byte array + /// Source string to evaluate /// Parsed long value /// True if input contained only ASCII decimal characters, otherwise false - static bool IsValidDouble(int length, byte* source, out double val) + static bool IsValidDouble(ReadOnlySpan source, out double val) { - val = 0; try { // Check for valid number - if (!NumUtils.TryReadDouble(length, source, out val) || !double.IsFinite(val)) - { - // Signal value is not a valid number - return false; - } + return NumUtils.TryReadDouble(source, out val) || !double.IsFinite(val); + } + catch + { + // Signal value is not a valid number + val = 0; + return false; + } + } + + /// + /// Parse ASCII byte array into double and validate that only contains ASCII decimal characters + /// + /// Source string to evaluate + /// Parsed long value + /// True if input contained only ASCII decimal characters, otherwise false + static bool IsValidDouble(byte* source, int sourceLen, out double val) + { + try + { + // Check for valid number + return NumUtils.TryReadDouble(sourceLen, source, out val) || !double.IsFinite(val); } catch { // Signal value is not a valid number + val = 0; return false; } - return true; } /// - public int GetRMWInitialValueLength(ref RawStringInput input) + public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref RawStringInput input) { var cmd = input.header.cmd; + var fieldInfo = new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = 0, + HasETag = input.header.CheckWithETagFlag() + }; switch (cmd) { case RespCommand.SETBIT: var bOffset = input.arg1; - return sizeof(int) + BitmapManager.Length(bOffset); + fieldInfo.ValueDataSize = BitmapManager.Length(bOffset); + return fieldInfo; case RespCommand.BITFIELD: case RespCommand.BITFIELD_RO: var bitFieldArgs = GetBitFieldArguments(ref input); - return sizeof(int) + BitmapManager.LengthFromType(bitFieldArgs); + fieldInfo.ValueDataSize = BitmapManager.LengthFromType(bitFieldArgs); + return fieldInfo; case RespCommand.PFADD: - return sizeof(int) + HyperLogLog.DefaultHLL.SparseInitialLength(ref input); + fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.SparseInitialLength(ref input); + return fieldInfo; case RespCommand.PFMERGE: - var length = input.parseState.GetArgSliceByRef(0).SpanByte.Length; - return sizeof(int) + length; + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + return fieldInfo; + + case RespCommand.SETIFGREATER: + case RespCommand.SETIFMATCH: + fieldInfo.ValueDataSize = 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.HasExpiration = input.arg1 != 0; + return fieldInfo; + + case RespCommand.SETKEEPTTL: + // Copy input to value; do not change expiration + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + return fieldInfo; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - return sizeof(int) + newValue.Length + offset; + fieldInfo.ValueDataSize = newValue.Length + offset; + return fieldInfo; case RespCommand.APPEND: var valueLength = input.parseState.GetArgSliceByRef(0).Length; - return sizeof(int) + valueLength; + fieldInfo.ValueDataSize = valueLength; + return fieldInfo; case RespCommand.INCR: - return sizeof(int) + 1; // # of digits in "1" + fieldInfo.ValueDataSize = 1; // # of digits in "1" + return fieldInfo; case RespCommand.DECR: - return sizeof(int) + 2; // # of digits in "-1" + fieldInfo.ValueDataSize = 2; // # of digits in "-1" + return fieldInfo; case RespCommand.INCRBY: var ndigits = NumUtils.CountDigits(input.arg1, out var isNegative); - return sizeof(int) + ndigits + (isNegative ? 1 : 0); + fieldInfo.ValueDataSize = ndigits + (isNegative ? 1 : 0); + return fieldInfo; case RespCommand.DECRBY: ndigits = NumUtils.CountDigits(-input.arg1, out isNegative); - return sizeof(int) + ndigits + (isNegative ? 1 : 0); - case RespCommand.INCRBYFLOAT: - if (!input.parseState.TryGetDouble(0, out var incrByFloat)) - return sizeof(int); + fieldInfo.ValueDataSize = ndigits + (isNegative ? 1 : 0); + return fieldInfo; - ndigits = NumUtils.CountCharsInDouble(incrByFloat, out var _, out var _, out var _); + case RespCommand.INCRBYFLOAT: + fieldInfo.ValueDataSize = input.parseState.TryGetDouble(0, out var incrByFloat) + ? NumUtils.CountCharsInDouble(incrByFloat, out var _, out var _, out var _) + : sizeof(int); + return fieldInfo; - return sizeof(int) + ndigits; default: if (cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)cmd); - // Compute metadata size for result - int metadataSize = input.arg1 switch - { - -1 => 0, - 0 => 0, - _ => 8, - }; - return sizeof(int) + metadataSize + functions.GetInitialLength(ref input); + fieldInfo.ValueDataSize = functions.GetInitialLength(ref input); } - - return sizeof(int) + input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length + (input.arg1 == 0 ? 0 : sizeof(long)) + this.functionsState.etagState.etagOffsetForVarlen; + else + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; + fieldInfo.HasETag = input.header.CheckWithETagFlag(); + fieldInfo.HasExpiration = input.arg1 != 0; + return fieldInfo; } } /// - public int GetRMWModifiedValueLength(ref SpanByte t, ref RawStringInput input) + public RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref RawStringInput input) + where TSourceLogRecord : ISourceLogRecord { + var fieldInfo = new RecordFieldInfo() + { + KeyDataSize = srcLogRecord.Key.Length, + ValueDataSize = 0, + HasETag = input.header.CheckWithETagFlag() || srcLogRecord.Info.HasETag, + HasExpiration = srcLogRecord.Info.HasExpiration + }; + if (input.header.cmd != RespCommand.NONE) { var cmd = input.header.cmd; @@ -146,127 +215,220 @@ public int GetRMWModifiedValueLength(ref SpanByte t, ref RawStringInput input) case RespCommand.INCRBY: var incrByValue = input.header.cmd == RespCommand.INCRBY ? input.arg1 : 1; - var curr = NumUtils.ReadInt64(t.AsSpan(functionsState.etagState.etagOffsetForVarlen)); - var next = curr + incrByValue; - - var ndigits = NumUtils.CountDigits(next, out var isNegative); - ndigits += isNegative ? 1 : 0; + var value = srcLogRecord.ValueSpan; + fieldInfo.ValueDataSize = 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 _)) + { + // TODO Consider adding a way to cache curr for the IPU call + var curr = NumUtils.ReadInt64(value); + var next = curr + incrByValue; - return sizeof(int) + ndigits + t.MetadataSize + functionsState.etagState.etagOffsetForVarlen; + fieldInfo.ValueDataSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); + } + return fieldInfo; case RespCommand.DECR: case RespCommand.DECRBY: var decrByValue = input.header.cmd == RespCommand.DECRBY ? input.arg1 : 1; - curr = NumUtils.ReadInt64(t.AsSpan(functionsState.etagState.etagOffsetForVarlen)); - next = curr - decrByValue; - - ndigits = NumUtils.CountDigits(next, out isNegative); - ndigits += isNegative ? 1 : 0; + value = srcLogRecord.ValueSpan; + fieldInfo.ValueDataSize = 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; - return sizeof(int) + ndigits + t.MetadataSize + functionsState.etagState.etagOffsetForVarlen; + fieldInfo.ValueDataSize = NumUtils.CountDigits(next, out var isNegative) + (isNegative ? 1 : 0); + } + return fieldInfo; case RespCommand.INCRBYFLOAT: // We don't need to TryGetDouble here because InPlaceUpdater will raise an error before we reach this point var incrByFloat = input.parseState.GetDouble(0); - NumUtils.TryReadDouble(t.AsSpan(functionsState.etagState.etagOffsetForVarlen), out var currVal); - var nextVal = currVal + incrByFloat; + value = srcLogRecord.ValueSpan; + fieldInfo.ValueDataSize = 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; - ndigits = NumUtils.CountCharsInDouble(nextVal, out _, out _, out _); + fieldInfo.ValueDataSize = NumUtils.CountCharsInDouble(nextVal, out _, out _, out _); + } + return fieldInfo; - return sizeof(int) + ndigits + t.MetadataSize + functionsState.etagState.etagOffsetForVarlen; case RespCommand.SETBIT: var bOffset = input.arg1; - return sizeof(int) + BitmapManager.NewBlockAllocLength(t.Length, bOffset); + fieldInfo.ValueDataSize = BitmapManager.NewBlockAllocLength(srcLogRecord.ValueSpan.Length, bOffset); + return fieldInfo; + case RespCommand.BITFIELD: case RespCommand.BITFIELD_RO: var bitFieldArgs = GetBitFieldArguments(ref input); - return sizeof(int) + BitmapManager.NewBlockAllocLengthFromType(bitFieldArgs, t.Length); + fieldInfo.ValueDataSize = BitmapManager.NewBlockAllocLengthFromType(bitFieldArgs, srcLogRecord.ValueSpan.Length); + return fieldInfo; + case RespCommand.PFADD: - var length = sizeof(int); - var v = t.ToPointer(); - length += HyperLogLog.DefaultHLL.UpdateGrow(ref input, v); - return length + t.MetadataSize; + // 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); + else + fixed(byte* valuePtr = srcLogRecord.ValueSpan) + fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.UpdateGrow(ref input, valuePtr); + return fieldInfo; case RespCommand.PFMERGE: - length = sizeof(int); - var srcHLL = input.parseState.GetArgSliceByRef(0).SpanByte.ToPointer(); - var dstHLL = t.ToPointer(); - length += HyperLogLog.DefaultHLL.MergeGrow(srcHLL, dstHLL); - return length + t.MetadataSize; + // 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); + else + fixed(byte* dstHLL = srcLogRecord.ValueSpan) + fieldInfo.ValueDataSize = HyperLogLog.DefaultHLL.MergeGrow(srcHLL, dstHLL); + return fieldInfo; case RespCommand.SETKEEPTTLXX: case RespCommand.SETKEEPTTL: - var setValue = input.parseState.GetArgSliceByRef(0); - return sizeof(int) + t.MetadataSize + setValue.Length + functionsState.etagState.etagOffsetForVarlen; + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + return fieldInfo; case RespCommand.SET: case RespCommand.SETEXXX: - return sizeof(int) + input.parseState.GetArgSliceByRef(0).Length + (input.arg1 == 0 ? 0 : sizeof(long)) + functionsState.etagState.etagOffsetForVarlen; + case RespCommand.SETEXNX: + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.HasExpiration = input.arg1 != 0; + return fieldInfo; + case RespCommand.PERSIST: - return sizeof(int) + t.LengthWithoutMetadata; + fieldInfo.HasExpiration = false; + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + return fieldInfo; + case RespCommand.SETIFGREATER: case RespCommand.SETIFMATCH: - var newValue = input.parseState.GetArgSliceByRef(0).ReadOnlySpan; - int metadataSize = input.arg1 == 0 ? t.MetadataSize : sizeof(long); - return sizeof(int) + newValue.Length + EtagConstants.EtagSize + metadataSize; + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).ReadOnlySpan.Length; + fieldInfo.HasETag = true; + fieldInfo.HasExpiration = input.arg1 != 0 || srcLogRecord.Info.HasExpiration; + return fieldInfo; + case RespCommand.EXPIRE: case RespCommand.PEXPIRE: case RespCommand.EXPIREAT: case RespCommand.PEXPIREAT: - return sizeof(int) + t.Length + sizeof(long); + { + // Set HasExpiration to match with EvaluateExpireInPlace. + var expireOption = (ExpireOption)input.arg1; + 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 + { + switch (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; + } + } + } + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + return fieldInfo; case RespCommand.SETRANGE: var offset = input.parseState.GetInt(0); - newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; + var newValue = input.parseState.GetArgSliceByRef(1).ReadOnlySpan; - if (newValue.Length + offset > t.LengthWithoutMetadata - functionsState.etagState.etagOffsetForVarlen) - return sizeof(int) + newValue.Length + offset + t.MetadataSize + functionsState.etagState.etagOffsetForVarlen; - return sizeof(int) + t.Length; + fieldInfo.ValueDataSize = newValue.Length + offset; + if (fieldInfo.ValueDataSize < srcLogRecord.ValueSpan.Length) + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + return fieldInfo; case RespCommand.GETEX: - return sizeof(int) + t.LengthWithoutMetadata + (input.arg1 > 0 ? sizeof(long) : 0); + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length; + + // If both EX and PERSIST were specified, EX wins + if (input.arg1 > 0) + fieldInfo.HasExpiration = true; + else if (input.parseState.Count > 0) + { + if (input.parseState.GetArgSliceByRef(0).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase(CmdStrings.PERSIST)) + fieldInfo.HasExpiration = false; + } + + return fieldInfo; case RespCommand.APPEND: - var valueLength = input.parseState.GetArgSliceByRef(0).Length; - return sizeof(int) + t.Length + valueLength; + fieldInfo.ValueDataSize = srcLogRecord.ValueSpan.Length + input.parseState.GetArgSliceByRef(0).Length; + return fieldInfo; case RespCommand.GETDEL: case RespCommand.DELIFGREATER: // Min allocation (only metadata) needed since this is going to be used for tombstoning anyway. - return sizeof(int); + return fieldInfo; default: if (cmd > RespCommandExtensions.LastValidCommand) { var functions = functionsState.GetCustomCommandFunctions((ushort)cmd); - // compute metadata for result - metadataSize = input.arg1 switch - { - -1 => 0, - 0 => t.MetadataSize, - _ => 8, - }; - return sizeof(int) + metadataSize + functions.GetLength(t.AsReadOnlySpan(), ref input); + fieldInfo.ValueDataSize = functions.GetLength(srcLogRecord.ValueSpan, ref input); + fieldInfo.HasExpiration = input.arg1 != 0; + return fieldInfo; } throw new GarnetException("Unsupported operation on input"); } } - return sizeof(int) + input.parseState.GetArgSliceByRef(0).Length + - (input.arg1 == 0 ? 0 : sizeof(long)); + fieldInfo.ValueDataSize = input.parseState.GetArgSliceByRef(0).Length; + fieldInfo.HasExpiration = input.arg1 != 0; + return fieldInfo; } - public int GetUpsertValueLength(ref SpanByte t, ref RawStringInput input) + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref RawStringInput input) { + var fieldInfo = new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = value.Length, + HasETag = input.header.CheckWithETagFlag() + }; + switch (input.header.cmd) { case RespCommand.SET: case RespCommand.SETEX: - return input.arg1 == 0 ? t.TotalSize : sizeof(int) + t.LengthWithoutMetadata + sizeof(long); + case RespCommand.APPEND: + fieldInfo.HasExpiration = input.arg1 != 0; + break; } + return fieldInfo; + } - return t.TotalSize; + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref RawStringInput input) + => throw new GarnetException("String store should not be called with IHeapObject"); + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref RawStringInput input) + where TSourceLogRecord : ISourceLogRecord + { + if (inputLogRecord.Info.ValueIsObject) + throw new GarnetException("String store should not be called with IHeapObject"); + return inputLogRecord.GetRecordFieldInfo(); } } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/ObjectStore/CallbackMethods.cs b/libs/server/Storage/Functions/ObjectStore/CallbackMethods.cs index 3765fb0bdb9..bd06be25133 100644 --- a/libs/server/Storage/Functions/ObjectStore/CallbackMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/CallbackMethods.cs @@ -8,15 +8,15 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public void ReadCompletionCallback(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) + public void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref ObjectInput input, ref GarnetObjectStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) { } /// - public void RMWCompletionCallback(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) + public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref ObjectInput input, ref GarnetObjectStoreOutput output, long ctx, Status status, RecordMetadata recordMetadata) { } } diff --git a/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs b/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs index 05bb422b47f..2db027602b7 100644 --- a/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/DeleteMethods.cs @@ -8,29 +8,29 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public bool SingleDeleter(ref byte[] key, ref IGarnetObject value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; /// - public void PostSingleDeleter(ref byte[] key, ref DeleteInfo deleteInfo) + public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - if (!deleteInfo.RecordInfo.Modified) + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogDelete(ref key, deleteInfo.Version, deleteInfo.SessionID); + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); } /// - public bool ConcurrentDeleter(ref byte[] key, ref IGarnetObject value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - if (!deleteInfo.RecordInfo.Modified) + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(deleteInfo.KeyHash); if (functionsState.appendOnlyFile != null) - WriteLogDelete(ref key, deleteInfo.Version, deleteInfo.SessionID); - functionsState.objectStoreSizeTracker?.AddTrackedSize(-value.Size); + WriteLogDelete(logRecord.Key, deleteInfo.Version, deleteInfo.SessionID); + functionsState.objectStoreSizeTracker?.AddTrackedSize(-logRecord.ValueObject.MemorySize); return true; } } diff --git a/libs/server/Storage/Functions/ObjectStore/ObjectSessionFunctions.cs b/libs/server/Storage/Functions/ObjectStore/ObjectSessionFunctions.cs index 2295eac2a76..c4887cb0c37 100644 --- a/libs/server/Storage/Functions/ObjectStore/ObjectSessionFunctions.cs +++ b/libs/server/Storage/Functions/ObjectStore/ObjectSessionFunctions.cs @@ -8,7 +8,7 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { readonly FunctionsState functionsState; diff --git a/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs b/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs index a1c143673fe..49db42c7a0b 100644 --- a/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/PrivateMethods.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System; -using System.Buffers; using System.Diagnostics; using System.Runtime.CompilerServices; using Garnet.common; @@ -13,149 +12,111 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// /// Logging upsert from - /// a. ConcurrentWriter - /// b. PostSingleWriter + /// a. InPlaceWriter + /// b. PostInitialWriter /// - void WriteLogUpsert(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, long version, int sessionID) + void WriteLogUpsert(ReadOnlySpan key, ref ObjectInput input, ReadOnlySpan value, long version, int sessionID) { - if (functionsState.StoredProcMode) return; + if (functionsState.StoredProcMode) + return; input.header.flags |= RespInputFlags.Deterministic; - var valueBytes = GarnetObjectSerializer.Serialize(value); - fixed (byte* ptr = key) - { - fixed (byte* valPtr = valueBytes) - { - var keySB = SpanByte.FromPinnedPointer(ptr, key.Length); - var valSB = SpanByte.FromPinnedPointer(valPtr, valueBytes.Length); - - functionsState.appendOnlyFile.Enqueue( - new AofHeader { opType = AofEntryType.ObjectStoreUpsert, storeVersion = version, sessionID = sessionID }, - ref keySB, ref valSB, out _); - } - } + functionsState.appendOnlyFile.Enqueue( + new AofHeader { opType = AofEntryType.ObjectStoreUpsert, storeVersion = version, sessionID = sessionID }, + key, value, out _); } /// - /// Logging RMW from - /// a. PostInitialUpdater - /// b. InPlaceUpdater - /// c. PostCopyUpdater + /// Logging upsert from + /// a. InPlaceWriter + /// b. PostInitialWriter /// - void WriteLogRMW(ref byte[] key, ref ObjectInput input, long version, int sessionID) + void WriteLogUpsert(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, long version, int sessionID) { - if (functionsState.StoredProcMode) return; + if (functionsState.StoredProcMode) + return; input.header.flags |= RespInputFlags.Deterministic; - // Serializing key & ObjectInput to RMW log - fixed (byte* keyPtr = key) + GarnetObjectSerializer.Serialize(value, out var valueBytes); + fixed (byte* valPtr = valueBytes) { - var sbKey = SpanByte.FromPinnedPointer(keyPtr, key.Length); - functionsState.appendOnlyFile.Enqueue( - new AofHeader { opType = AofEntryType.ObjectStoreRMW, storeVersion = version, sessionID = sessionID }, - ref sbKey, ref input, out _); + new AofHeader { opType = AofEntryType.ObjectStoreUpsert, storeVersion = version, sessionID = sessionID }, + key, new ReadOnlySpan(valPtr, valueBytes.Length), out _); } } /// - /// Logging Delete from - /// a. ConcurrentDeleter - /// b. PostSingleDeleter + /// Logging RMW from + /// a. PostInitialUpdater + /// b. InPlaceUpdater + /// c. PostCopyUpdater /// - void WriteLogDelete(ref byte[] key, long version, int sessionID) + void WriteLogRMW(ReadOnlySpan key, ref ObjectInput input, long version, int sessionID) { if (functionsState.StoredProcMode) return; - fixed (byte* ptr = key) - { - var keySB = SpanByte.FromPinnedPointer(ptr, key.Length); - SpanByte valSB = default; + input.header.flags |= RespInputFlags.Deterministic; - functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.ObjectStoreDelete, storeVersion = version, sessionID = sessionID }, ref keySB, ref valSB, out _); - } + functionsState.appendOnlyFile.Enqueue( + new AofHeader { opType = AofEntryType.ObjectStoreRMW, storeVersion = version, sessionID = sessionID }, + key, ref input, out _); } - internal static bool CheckExpiry(IGarnetObject src) => src.Expiration < DateTimeOffset.UtcNow.Ticks; - - static void CopyRespNumber(long number, ref SpanByteAndMemory dst) + /// + /// Logging Delete from + /// a. InPlaceDeleter + /// b. PostInitialDeleter + /// + void WriteLogDelete(ReadOnlySpan key, long version, int sessionID) { - byte* curr = dst.SpanByte.ToPointer(); - byte* end = curr + dst.SpanByte.Length; - if (RespWriteUtils.TryWriteInt64(number, ref curr, end, out var integerLen, out int totalLen)) - { - dst.SpanByte.Length = (int)(curr - dst.SpanByte.ToPointer()); + if (functionsState.StoredProcMode) return; - } - //handle resp buffer overflow here - dst.ConvertToHeap(); - dst.Length = totalLen; - dst.Memory = MemoryPool.Shared.Rent(totalLen); - fixed (byte* ptr = dst.Memory.Memory.Span) - { - byte* cc = ptr; - *cc++ = (byte)':'; - NumUtils.WriteInt64(number, (int)integerLen, ref cc); - *cc++ = (byte)'\r'; - *cc++ = (byte)'\n'; - } + functionsState.appendOnlyFile.Enqueue(new AofHeader { opType = AofEntryType.ObjectStoreDelete, storeVersion = version, sessionID = sessionID }, key, item2: default, out _); } - static void CopyDefaultResp(ReadOnlySpan resp, ref SpanByteAndMemory dst) - { - if (resp.Length < dst.SpanByte.Length) - { - resp.CopyTo(dst.SpanByte.AsSpan()); - dst.SpanByte.Length = resp.Length; - return; - } - - dst.ConvertToHeap(); - dst.Length = resp.Length; - dst.Memory = MemoryPool.Shared.Rent(resp.Length); - resp.CopyTo(dst.Memory.Memory.Span); - } + internal static bool CheckExpiry(ref TSourceLogRecord srcLogRecord) + where TSourceLogRecord : ISourceLogRecord + => srcLogRecord.Info.HasExpiration && srcLogRecord.Expiration < DateTimeOffset.UtcNow.Ticks; - static bool EvaluateObjectExpireInPlace(ExpireOption optionType, bool expiryExists, long expiration, ref IGarnetObject value, ref GarnetObjectStoreOutput output) + 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(); - if (expiryExists) + o->result1 = 0; + if (logRecord.Info.HasExpiration) { switch (optionType) { case ExpireOption.NX: - o->result1 = 0; - break; + return true; case ExpireOption.XX: case ExpireOption.None: - value.Expiration = expiration; + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + return true; case ExpireOption.GT: case ExpireOption.XXGT: - bool replace = expiration < value.Expiration; - value.Expiration = replace ? value.Expiration : expiration; - if (replace) - o->result1 = 0; - else + if (newExpiry > logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; case ExpireOption.LT: case ExpireOption.XXLT: - replace = expiration > value.Expiration; - value.Expiration = replace ? value.Expiration : expiration; - if (replace) - o->result1 = 0; - else + if (newExpiry < logRecord.Expiration) + { + _ = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + } + return true; default: - throw new GarnetException($"EvaluateObjectExpireInPlace exception expiryExists:{expiryExists}, optionType{optionType}"); + throw new GarnetException($"EvaluateObjectExpireInPlace exception expiryExists: True, optionType {optionType}"); } } else @@ -164,21 +125,19 @@ static bool EvaluateObjectExpireInPlace(ExpireOption optionType, bool expiryExis { case ExpireOption.NX: case ExpireOption.None: - case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite - value.Expiration = expiration; + case ExpireOption.LT: // If expiry doesn't exist, LT should treat the current expiration as infinite, so the new value must be less + var ok = logRecord.TrySetExpiration(newExpiry); o->result1 = 1; - break; + return ok; case ExpireOption.XX: - case ExpireOption.GT: + 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: - o->result1 = 0; - break; + return true; default: - throw new GarnetException($"EvaluateObjectExpireInPlace exception expiryExists:{expiryExists}, optionType{optionType}"); + throw new GarnetException($"EvaluateObjectExpireInPlace exception expiryExists: False, optionType {optionType}"); } } - return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs index 82b3e1699f0..0b38973ad0d 100644 --- a/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/RMWMethods.cs @@ -12,10 +12,10 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public bool NeedInitialUpdate(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + public bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) { var type = input.header.type; @@ -41,63 +41,69 @@ public bool NeedInitialUpdate(ref byte[] key, ref ObjectInput input, ref GarnetO } /// - public bool InitialUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) { + Debug.Assert(!logRecord.Info.HasETag && !logRecord.Info.HasExpiration, "Should not have Expiration or ETag on InitialUpdater log records"); + var type = input.header.type; + IGarnetObject value; if ((byte)type < CustomCommandManager.CustomTypeIdStartOffset) { value = GarnetObject.Create(type); - value.Operate(ref input, ref output, functionsState.respProtocolVersion, out _); + _ = value.Operate(ref input, ref output, functionsState.respProtocolVersion, out _); + _ = logRecord.TrySetValueObject(value, ref sizeInfo); return true; } - else - { - Debug.Assert(type != GarnetObjectType.Expire && type != GarnetObjectType.PExpire && type != GarnetObjectType.Persist, "Expire and Persist commands should have been handled already by NeedInitialUpdate."); - var customObjectCommand = GetCustomObjectCommand(ref input, type); - value = functionsState.GetCustomObjectFactory((byte)type).Create((byte)type); + 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."); - (IMemoryOwner Memory, int Length) outp = (output.SpanByteAndMemory.Memory, 0); - var result = customObjectCommand.InitialUpdater(key, ref input, value, ref outp, ref rmwInfo); - output.SpanByteAndMemory.Memory = outp.Memory; - output.SpanByteAndMemory.Length = outp.Length; - return result; - } + var customObjectCommand = GetCustomObjectCommand(ref input, type); + value = functionsState.GetCustomObjectFactory((byte)type).Create((byte)type); + + (IMemoryOwner Memory, int Length) memoryAndLength = (output.SpanByteAndMemory.Memory, 0); + var result = customObjectCommand.InitialUpdater(logRecord.Key, ref input, value, ref memoryAndLength, ref rmwInfo); + _ = logRecord.TrySetValueObject(value, ref sizeInfo); + output.SpanByteAndMemory.Memory = memoryAndLength.Memory; + output.SpanByteAndMemory.Length = memoryAndLength.Length; + if (result) + sizeInfo.AssertOptionals(logRecord.Info); + return result; } /// - public void PostInitialUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + public void PostInitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) { functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); if (functionsState.appendOnlyFile != null) { input.header.SetExpiredFlag(); - WriteLogRMW(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + WriteLogRMW(dstLogRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); } - functionsState.objectStoreSizeTracker?.AddTrackedSize(MemoryUtils.CalculateKeyValueSize(key, value)); + functionsState.objectStoreSizeTracker?.AddTrackedSize(dstLogRecord.ValueObject.MemorySize); } /// - public bool InPlaceUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) { - if (InPlaceUpdaterWorker(ref key, ref input, ref value, ref output, ref rmwInfo, out var sizeChange)) + if (InPlaceUpdaterWorker(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo, out long sizeChange)) { - if (!rmwInfo.RecordInfo.Modified) + if (!logRecord.Info.Modified) functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); - if (functionsState.appendOnlyFile != null) WriteLogRMW(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + if (functionsState.appendOnlyFile != null) + WriteLogRMW(logRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); functionsState.objectStoreSizeTracker?.AddTrackedSize(sizeChange); return true; } return false; } - bool InPlaceUpdaterWorker(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo, out long sizeChange) + bool InPlaceUpdaterWorker(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo, out long sizeChange) { sizeChange = 0; // Expired data - if (value.Expiration > 0 && input.header.CheckExpiry(value.Expiration)) + if (logRecord.Info.HasExpiration && input.header.CheckExpiry(logRecord.Expiration)) { rmwInfo.Action = RMWAction.ExpireAndResume; return false; @@ -127,60 +133,66 @@ bool InPlaceUpdaterWorker(ref byte[] key, ref ObjectInput input, ref IGarnetObje expiryTicks = DateTimeOffset.UtcNow.Ticks + tsExpiry.Ticks; } - var expiryExists = value.Expiration > 0; - return EvaluateObjectExpireInPlace(optionType, expiryExists, expiryTicks, ref value, ref output); + if (!EvaluateObjectExpireInPlace(ref logRecord, optionType, expiryTicks, 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 (value.Expiration > 0) + if (logRecord.Info.HasExpiration) { - value.Expiration = 0; - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); + logRecord.RemoveExpiration(); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); } else - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); return true; default: if ((byte)input.header.type < CustomCommandManager.CustomTypeIdStartOffset) { - var operateSuccessful = value.Operate(ref input, ref output, functionsState.respProtocolVersion, out sizeChange); + var operateSuccessful = ((IGarnetObject)logRecord.ValueObject).Operate(ref input, ref output, functionsState.respProtocolVersion, out sizeChange); if (output.HasWrongType) return true; - if (output.HasRemoveKey) { rmwInfo.Action = RMWAction.ExpireAndStop; return false; } + sizeInfo.AssertOptionals(logRecord.Info); return operateSuccessful; } else { - if (IncorrectObjectType(ref input, value, ref output.SpanByteAndMemory)) + if (IncorrectObjectType(ref input, ((IGarnetObject)logRecord.ValueObject), ref output.SpanByteAndMemory)) { output.OutputFlags |= ObjectStoreOutputFlags.WrongType; return true; } - (IMemoryOwner Memory, int Length) outp = (output.SpanByteAndMemory.Memory, 0); + (IMemoryOwner Memory, int Length) memoryAndLength = (output.SpanByteAndMemory.Memory, 0); var customObjectCommand = GetCustomObjectCommand(ref input, input.header.type); - var result = customObjectCommand.Updater(key, ref input, value, ref outp, ref rmwInfo); - output.SpanByteAndMemory.Memory = outp.Memory; - output.SpanByteAndMemory.Length = outp.Length; - return result; - //return customObjectCommand.InPlaceUpdateWorker(key, ref input, value, ref output.spanByteAndMemory, ref rmwInfo); + var result = customObjectCommand.Updater(logRecord.Key, ref input, ((IGarnetObject)logRecord.ValueObject), ref memoryAndLength, ref rmwInfo); + output.SpanByteAndMemory.Memory = memoryAndLength.Memory; + output.SpanByteAndMemory.Length = memoryAndLength.Length; + if (!result) + return false; + break; } } + sizeInfo.AssertOptionals(logRecord.Info); + return true; } /// - public bool NeedCopyUpdate(ref byte[] key, ref ObjectInput input, ref IGarnetObject oldValue, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + public bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord => true; /// - public bool CopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject oldValue, ref IGarnetObject newValue, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { // Expired data - if (oldValue.Expiration > 0 && input.header.CheckExpiry(oldValue.Expiration)) + if (srcLogRecord.Info.HasExpiration && input.header.CheckExpiry(srcLogRecord.Expiration)) { rmwInfo.Action = RMWAction.ExpireAndResume; return false; @@ -189,12 +201,18 @@ public bool CopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject } /// - public bool PostCopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetObject oldValue, ref IGarnetObject value, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + public bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { // 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 = oldValue.Size; - oldValue.CopyUpdate(ref oldValue, ref value, rmwInfo.RecordInfo.IsInNewVersion); + 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, ref sizeInfo)) + return false; functionsState.watchVersionMap.IncrementVersion(rmwInfo.KeyHash); @@ -222,18 +240,23 @@ public bool PostCopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetOb expiryTicks = DateTimeOffset.UtcNow.Ticks + tsExpiry.Ticks; } - var expiryExists = value.Expiration > 0; - - EvaluateObjectExpireInPlace(optionType, expiryExists, expiryTicks, ref value, ref output); + // 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, optionType, expiryTicks, ref output)) + return false; break; + case GarnetObjectType.Persist: - if (value.Expiration > 0) + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; + if (srcLogRecord.Info.HasExpiration) { - value.Expiration = 0; - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); + dstLogRecord.RemoveExpiration(); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_1, ref output.SpanByteAndMemory); } else - CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); + functionsState.CopyDefaultResp(CmdStrings.RESP_RETURN_VAL_0, ref output.SpanByteAndMemory); break; default: if ((byte)input.header.type < CustomCommandManager.CustomTypeIdStartOffset) @@ -241,7 +264,6 @@ public bool PostCopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetOb value.Operate(ref input, ref output, functionsState.respProtocolVersion, out _); if (output.HasWrongType) return true; - if (output.HasRemoveKey) { rmwInfo.Action = RMWAction.ExpireAndStop; @@ -261,19 +283,20 @@ public bool PostCopyUpdater(ref byte[] key, ref ObjectInput input, ref IGarnetOb (IMemoryOwner Memory, int Length) outp = (output.SpanByteAndMemory.Memory, 0); var customObjectCommand = GetCustomObjectCommand(ref input, input.header.type); - var result = customObjectCommand.Updater(key, ref input, value, ref outp, ref rmwInfo); + var result = customObjectCommand.Updater(srcLogRecord.Key, ref input, value, ref outp, ref rmwInfo); output.SpanByteAndMemory.Memory = outp.Memory; output.SpanByteAndMemory.Length = outp.Length; return result; } } + sizeInfo.AssertOptionals(dstLogRecord.Info); - // If oldValue has been set to null, subtract it's size from the tracked heap size - var sizeAdjustment = oldValue == null ? value.Size - oldValueSize : value.Size; + // 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(ref key, ref input, rmwInfo.Version, rmwInfo.SessionID); + WriteLogRMW(srcLogRecord.Key, ref input, rmwInfo.Version, rmwInfo.SessionID); return true; } } diff --git a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs index f6abd17caf3..1b82a451036 100644 --- a/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/ReadMethods.cs @@ -11,12 +11,13 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public bool SingleReader(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput dst, ref ReadInfo readInfo) + public bool Reader(ref TSourceLogRecord srcLogRecord, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord { - if (value.Expiration > 0 && value.Expiration < DateTimeOffset.Now.UtcTicks) + if (srcLogRecord.Info.HasExpiration && srcLogRecord.Expiration < DateTimeOffset.Now.UtcTicks) { // Do not set 'value = null' or otherwise mark this; Reads should not update the database. We rely on consistently checking for expiration everywhere. readInfo.Action = ReadAction.Expire; @@ -27,55 +28,54 @@ public bool SingleReader(ref byte[] key, ref ObjectInput input, ref IGarnetObjec { switch (input.header.type) { + case GarnetObjectType.Migrate: + DiskLogRecord.Serialize(ref srcLogRecord, functionsState.garnetObjectSerializer, ref output.SpanByteAndMemory, functionsState.memoryPool); + return true; case GarnetObjectType.Ttl: - var ttlValue = ConvertUtils.SecondsFromDiffUtcNowTicks(value.Expiration > 0 ? value.Expiration : -1); - CopyRespNumber(ttlValue, ref dst.SpanByteAndMemory); + var ttlValue = ConvertUtils.SecondsFromDiffUtcNowTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(ttlValue, ref output.SpanByteAndMemory); return true; case GarnetObjectType.PTtl: - ttlValue = ConvertUtils.MillisecondsFromDiffUtcNowTicks(value.Expiration > 0 ? value.Expiration : -1); - CopyRespNumber(ttlValue, ref dst.SpanByteAndMemory); + ttlValue = ConvertUtils.MillisecondsFromDiffUtcNowTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(ttlValue, ref output.SpanByteAndMemory); return true; case GarnetObjectType.ExpireTime: - var expireTime = ConvertUtils.UnixTimeInSecondsFromTicks(value.Expiration > 0 ? value.Expiration : -1); - CopyRespNumber(expireTime, ref dst.SpanByteAndMemory); + var expireTime = ConvertUtils.UnixTimeInSecondsFromTicks(srcLogRecord.Info.HasExpiration ? srcLogRecord.Expiration : -1); + functionsState.CopyRespNumber(expireTime, ref output.SpanByteAndMemory); return true; case GarnetObjectType.PExpireTime: - expireTime = ConvertUtils.UnixTimeInMillisecondsFromTicks(value.Expiration > 0 ? value.Expiration : -1); - CopyRespNumber(expireTime, ref dst.SpanByteAndMemory); + 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) { - var opResult = value.Operate(ref input, ref dst, functionsState.respProtocolVersion, out _); - if (dst.HasWrongType) + var opResult = ((IGarnetObject)srcLogRecord.ValueObject).Operate(ref input, ref output, functionsState.respProtocolVersion, out _); + if (output.HasWrongType) return true; return opResult; } - if (IncorrectObjectType(ref input, value, ref dst.SpanByteAndMemory)) + if (IncorrectObjectType(ref input, (IGarnetObject)srcLogRecord.ValueObject, ref output.SpanByteAndMemory)) { - dst.OutputFlags |= ObjectStoreOutputFlags.WrongType; + output.OutputFlags |= ObjectStoreOutputFlags.WrongType; return true; } - (IMemoryOwner Memory, int Length) outp = (dst.SpanByteAndMemory.Memory, 0); + (IMemoryOwner Memory, int Length) outp = (output.SpanByteAndMemory.Memory, 0); var customObjectCommand = GetCustomObjectCommand(ref input, input.header.type); - var result = customObjectCommand.Reader(key, ref input, value, ref outp, ref readInfo); - dst.SpanByteAndMemory.Memory = outp.Memory; - dst.SpanByteAndMemory.Length = outp.Length; + var result = customObjectCommand.Reader(srcLogRecord.Key, ref input, (IGarnetObject)srcLogRecord.ValueObject, ref outp, ref readInfo); + output.SpanByteAndMemory.Memory = outp.Memory; + output.SpanByteAndMemory.Length = outp.Length; return result; } } - dst.GarnetObject = value; + output.GarnetObject = (IGarnetObject)srcLogRecord.ValueObject; return true; } - - /// - public bool ConcurrentReader(ref byte[] key, ref ObjectInput input, ref IGarnetObject value, ref GarnetObjectStoreOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - => SingleReader(ref key, ref input, ref value, ref dst, ref readInfo); } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs b/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs index 0d150e7cefb..883496d3614 100644 --- a/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/UpsertMethods.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using Tsavorite.core; namespace Garnet.server @@ -8,38 +9,155 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public bool SingleWriter(ref byte[] key, ref ObjectInput input, ref IGarnetObject src, ref IGarnetObject dst, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ReadOnlySpan srcValue, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) { - dst = src; + if (!dstLogRecord.TrySetValueSpan(srcValue, ref sizeInfo)) + return false; + // TODO ETag + if (input.arg1 != 0 && !dstLogRecord.TrySetExpiration(input.arg1)) + return false; + sizeInfo.AssertOptionals(dstLogRecord.Info); return true; } /// - public void PostSingleWriter(ref byte[] key, ref ObjectInput input, ref IGarnetObject src, ref IGarnetObject dst, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo, WriteReason reason) + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, IHeapObject srcValue, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) { - if (reason != WriteReason.CopyToTail) + if (!dstLogRecord.TrySetValueObject(srcValue, ref sizeInfo)) + return false; + // TODO ETag + if (input.arg1 != 0 && !dstLogRecord.TrySetExpiration(input.arg1)) + return false; + sizeInfo.AssertOptionals(dstLogRecord.Info); + return true; + } + + /// + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref TSourceLogRecord inputLogRecord, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + => dstLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ReadOnlySpan srcValue, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) + { + functionsState.watchVersionMap.IncrementVersion(upsertInfo.KeyHash); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, srcValue, 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 + // TODO: Need to track lengths written to readcache, which is now internal in Tsavorite + if (logRecord.Info.ValueIsOverflow) + functionsState.objectStoreSizeTracker?.AddTrackedSize(srcValue.Length); + } + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, IHeapObject srcValue, ref GarnetObjectStoreOutput 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(srcValue.MemorySize); + } + + /// + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref TSourceLogRecord inputLogRecord, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + 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); + } + + // TODO: Need to track original length as well, if it was overflow, and add overflow here as well as object size + var size = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + functionsState.objectStoreSizeTracker?.AddTrackedSize(size); + } + + /// + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ReadOnlySpan srcValue, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) + { + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TrySetValueSpan(srcValue, ref 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 (reason == WriteReason.Upsert && functionsState.appendOnlyFile != null) - WriteLogUpsert(ref key, ref input, ref src, upsertInfo.Version, upsertInfo.SessionID); + if (functionsState.appendOnlyFile != null) + WriteLogUpsert(logRecord.Key, ref input, srcValue, upsertInfo.Version, upsertInfo.SessionID); - if (reason == WriteReason.CopyToReadCache) - functionsState.objectStoreSizeTracker?.AddReadCacheTrackedSize(MemoryUtils.CalculateKeyValueSize(key, src)); - else - functionsState.objectStoreSizeTracker?.AddTrackedSize(MemoryUtils.CalculateKeyValueSize(key, src)); + // 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(srcValue.Length - oldSize); + return true; } /// - public bool ConcurrentWriter(ref byte[] key, ref ObjectInput input, ref IGarnetObject src, ref IGarnetObject dst, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, IHeapObject srcValue, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) { - dst = src; - if (!upsertInfo.RecordInfo.Modified) + var garnetObject = (IGarnetObject)srcValue; + + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TrySetValueObject(srcValue, ref 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(ref key, ref input, ref src, upsertInfo.Version, upsertInfo.SessionID); - functionsState.objectStoreSizeTracker?.AddTrackedSize(dst.Size - src.Size); + WriteLogUpsert(logRecord.Key, ref input, garnetObject, upsertInfo.Version, upsertInfo.SessionID); + + functionsState.objectStoreSizeTracker?.AddTrackedSize(srcValue.MemorySize - oldSize); + return true; + } + + /// + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref ObjectInput input, ref TSourceLogRecord inputLogRecord, ref GarnetObjectStoreOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + var oldSize = logRecord.Info.ValueIsInline + ? 0 + : (!logRecord.Info.ValueIsObject ? logRecord.ValueSpan.Length : logRecord.ValueObject.MemorySize); + + _ = logRecord.TryCopyFrom(ref inputLogRecord, ref 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); + } + + 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/ObjectStore/VarLenInputMethods.cs b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs index 63af04149be..9eca0b62942 100644 --- a/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs +++ b/libs/server/Storage/Functions/ObjectStore/VarLenInputMethods.cs @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using Garnet.common; +using System; using Tsavorite.core; namespace Garnet.server @@ -9,23 +9,85 @@ namespace Garnet.server /// /// Object store functions /// - public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions + public readonly unsafe partial struct ObjectSessionFunctions : ISessionFunctions { /// - public int GetRMWModifiedValueLength(ref IGarnetObject value, ref ObjectInput input) + public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref ObjectInput input) { - throw new GarnetException("GetRMWModifiedValueLength is not available on the object store"); + return new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = 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. + }; } /// - public int GetRMWInitialValueLength(ref ObjectInput input) + public RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref ObjectInput input) + where TSourceLogRecord : ISourceLogRecord { - throw new GarnetException("GetRMWInitialValueLength is not available on the object store"); + var fieldInfo = new RecordFieldInfo() + { + KeyDataSize = srcLogRecord.Key.Length, + ValueDataSize = ObjectIdMap.ObjectIdSize, + ValueIsObject = true, + 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) + { + return new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = value.Length, + ValueIsObject = false, + HasETag = false // TODO ETag not supported in Object store yet: input.header.CheckWithETagFlag() + // No object commands take an Expiration for Upsert. + }; + } + + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref ObjectInput input) + { + return new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = 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. + }; } - public int GetUpsertValueLength(ref IGarnetObject value, ref ObjectInput input) + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref ObjectInput input) + where TSourceLogRecord : ISourceLogRecord { - throw new GarnetException("GetUpsertInitialValueLength is not available on the object store"); + return new RecordFieldInfo() + { + KeyDataSize = key.Length, + ValueDataSize = 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. + }; } } } \ No newline at end of file diff --git a/libs/server/Storage/Functions/SimpleGarnetObjectSessionFunctions.cs b/libs/server/Storage/Functions/SimpleGarnetObjectSessionFunctions.cs new file mode 100644 index 00000000000..9e5215a15c7 --- /dev/null +++ b/libs/server/Storage/Functions/SimpleGarnetObjectSessionFunctions.cs @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using Tsavorite.core; + +namespace Garnet.server +{ + /// + /// Default simple functions base class with TInput and TOutput both being IGarnetObject; it assumes the Value is always an Object, never a Span. + /// + public class SimpleGarnetObjectSessionFunctions : SessionFunctionsBase + { + /// + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref IGarnetObject input, ref IGarnetObject output, ref ReadInfo readInfo) + { + if (srcLogRecord.Info.ValueIsObject) + { + output = (IGarnetObject)srcLogRecord.ValueObject; + return true; + } + return false; // TODO: possibly create an IGarnetObject from the serialized bytes + } + + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, IHeapObject srcValue, ref IGarnetObject output, ref UpsertInfo upsertInfo) + { + var result = base.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + if (result) + output = (IGarnetObject)srcValue; + return result; + } + + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, IHeapObject srcValue, ref IGarnetObject output, ref UpsertInfo upsertInfo) + { + var result = base.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + if (result) + output = (IGarnetObject)srcValue; + return result; + } + + /// + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, ref IGarnetObject output, ref RMWInfo rmwInfo) + { + var result = dstLogRecord.TrySetValueObject(input, ref sizeInfo); + if (result) + output = input; + return result; + } + /// + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, ref IGarnetObject output, ref RMWInfo rmwInfo) + { + // Simple base implementation does not use upsertInfo + var upsertInfo = new UpsertInfo(); + return base.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, ref srcLogRecord, ref output, ref upsertInfo); + } + /// + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, ref IGarnetObject output, ref RMWInfo rmwInfo) + { + // Simple base implementation does not use upsertInfo + var upsertInfo = new UpsertInfo(); + return InPlaceWriter(ref logRecord, ref sizeInfo, ref input, input, ref output, ref upsertInfo); + } + } +} diff --git a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs index 34194c3b351..b7df99f6b59 100644 --- a/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs +++ b/libs/server/Storage/Session/Common/ArrayKeyIterationFunctions.cs @@ -37,7 +37,7 @@ sealed partial class StorageSession : IDisposable /// size of every block or keys to return /// The type object to filter out /// - internal unsafe bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out long storeCursor, out List keys, long count = 10, ReadOnlySpan typeObject = default) + 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 << 49; Keys ??= new(); @@ -75,7 +75,7 @@ internal unsafe bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out lo } } - byte* patternPtr = patternB.ptr; + byte* patternPtr = patternB.ToPointer(); mainStoreDbScanFuncs ??= new(); mainStoreDbScanFuncs.Initialize(Keys, allKeys ? null : patternPtr, patternB.Length); @@ -117,13 +117,13 @@ internal unsafe bool DbScan(ArgSlice patternB, bool allKeys, long cursor, out lo /// /// internal bool IterateMainStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => basicContext.Session.IterateLookup(ref scanFunctions, untilAddress); /// /// Iterate the contents of the main store (pull based) /// - internal ITsavoriteScanIterator IterateMainStore() + internal ITsavoriteScanIterator IterateMainStore() => basicContext.Session.Iterate(); /// @@ -134,35 +134,34 @@ internal ITsavoriteScanIterator IterateMainStore() /// /// internal bool IterateObjectStore(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => objectStoreBasicContext.Session.IterateLookup(ref scanFunctions, untilAddress); /// /// Iterate the contents of the main store (pull based) /// - internal ITsavoriteScanIterator IterateObjectStore() + internal ITsavoriteScanIterator IterateObjectStore() => objectStoreBasicContext.Session.Iterate(); /// - /// Get a list of the keys in the store and object store - /// when using pattern + /// Get a list of the keys in the store and object store when using pattern /// /// - internal unsafe List DBKeys(ArgSlice pattern) + internal unsafe List DBKeys(PinnedSpanByte pattern) { Keys ??= new(); Keys.Clear(); - var allKeys = *pattern.ptr == '*' && pattern.Length == 1; + var allKeys = *pattern.ToPointer() == '*' && pattern.Length == 1; mainStoreDbKeysFuncs ??= new(); - mainStoreDbKeysFuncs.Initialize(Keys, allKeys ? null : pattern.ptr, pattern.Length); + 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.ptr, pattern.Length, matchType: null); + objStoreDbKeysFuncs.Initialize(Keys, allKeys ? null : pattern.ToPointer(), pattern.Length, matchType: null); objectStoreBasicContext.Session.Iterate(ref objStoreDbKeysFuncs); } @@ -211,7 +210,7 @@ internal void Initialize(List keys, byte* patternB, int length, Type mat } } - internal sealed class MainStoreGetDBKeys : IScanIteratorFunctions + internal sealed class MainStoreGetDBKeys : IScanIteratorFunctions { private readonly GetDBKeysInfo info; @@ -220,21 +219,34 @@ internal sealed class MainStoreGetDBKeys : IScanIteratorFunctions keys, byte* patternB, int length) => info.Initialize(keys, patternB, length); - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => ConcurrentReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { - if ((info.patternB != null && !GlobUtils.Match(info.patternB, info.patternLength, key.ToPointer(), key.Length, true)) - || (value.MetadataSize != 0 && MainSessionFunctions.CheckExpiry(ref value))) + var key = logRecord.Key; + + if (MainSessionFunctions.CheckExpiry(ref logRecord)) { cursorRecordResult = CursorRecordResult.Skip; + return true; } - else + + if (info.patternB != null) { - cursorRecordResult = CursorRecordResult.Accept; - info.keys.Add(key.ToByteArray()); + 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; } @@ -243,7 +255,7 @@ public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } } - internal sealed class ObjectStoreGetDBKeys : IScanIteratorFunctions + internal sealed class ObjectStoreGetDBKeys : IScanIteratorFunctions { private readonly GetDBKeysInfo info; @@ -252,36 +264,38 @@ internal sealed class ObjectStoreGetDBKeys : IScanIteratorFunctions keys, byte* patternB, int length, Type matchType = null) => info.Initialize(keys, patternB, length, matchType); - public bool SingleReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => ConcurrentReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - - public bool ConcurrentReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { - if (value.Expiration > 0 && ObjectSessionFunctions.CheckExpiry(value)) + if (ObjectSessionFunctions.CheckExpiry(ref logRecord)) { cursorRecordResult = CursorRecordResult.Skip; return true; } + var key = logRecord.Key; if (info.patternB != null) { - fixed (byte* keyPtr = key) + 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) { - if (!GlobUtils.Match(info.patternB, info.patternLength, keyPtr, key.Length, true)) - { - cursorRecordResult = CursorRecordResult.Skip; - return true; - } + cursorRecordResult = CursorRecordResult.Skip; + return true; } } - if (info.matchType != null && value.GetType() != info.matchType) + if (info.matchType != null && logRecord.ValueObject.GetType() != info.matchType) { cursorRecordResult = CursorRecordResult.Skip; return true; } - info.keys.Add(key); + info.keys.Add(key.ToArray()); cursorRecordResult = CursorRecordResult.Accept; return true; } @@ -299,7 +313,7 @@ internal class GetDBSizeInfo internal void Initialize() => count = 0; } - internal sealed class MainStoreGetDBSize : IScanIteratorFunctions + internal sealed class MainStoreGetDBSize : IScanIteratorFunctions { private readonly GetDBSizeInfo info; @@ -309,23 +323,23 @@ internal sealed class MainStoreGetDBSize : IScanIteratorFunctions info.Initialize(); - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Skip; - if (value.MetadataSize == 0 || !MainSessionFunctions.CheckExpiry(ref value)) + if (!MainSessionFunctions.CheckExpiry(ref logRecord)) { ++info.count; } return true; } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + 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 ObjectStoreGetDBSize : IScanIteratorFunctions { private readonly GetDBSizeInfo info; @@ -335,17 +349,17 @@ internal sealed class ObjectStoreGetDBSize : IScanIteratorFunctions info.Initialize(); - public bool SingleReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Skip; - if (value.Expiration == 0 || !ObjectSessionFunctions.CheckExpiry(value)) + if (!ObjectSessionFunctions.CheckExpiry(ref logRecord)) { ++info.count; } return true; } - public bool ConcurrentReader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); + public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } public void OnException(Exception exception, long numberOfRecords) { } diff --git a/libs/server/Storage/Session/Common/MemoryUtils.cs b/libs/server/Storage/Session/Common/MemoryUtils.cs index 7d76ad3a8ed..0b93ec85464 100644 --- a/libs/server/Storage/Session/Common/MemoryUtils.cs +++ b/libs/server/Storage/Session/Common/MemoryUtils.cs @@ -46,10 +46,8 @@ public static class MemoryUtils internal static long CalculateKeyValueSize(byte[] key, IGarnetObject value) { - // Round up key size to account for alignment during allocation - // and add up overhead for allocating a byte array - return Utility.RoundUp(key.Length, IntPtr.Size) + ByteArrayOverhead + - value.Size; + // Round up key size to account for alignment during allocation and add overhead for allocating a byte array + return Utility.RoundUp(key.Length, IntPtr.Size) + ByteArrayOverhead + value.MemorySize; } } } \ No newline at end of file diff --git a/libs/server/Storage/Session/MainStore/AdvancedOps.cs b/libs/server/Storage/Session/MainStore/AdvancedOps.cs index 828ec16e2a0..5dbcbfd6dee 100644 --- a/libs/server/Storage/Session/MainStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/MainStore/AdvancedOps.cs @@ -8,15 +8,15 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { - public GarnetStatus GET_WithPending(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus GET_WithPending(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.Read(ref key, ref input, ref output, ctx); + var status = context.Read(key, ref input, ref output, ctx); if (status.IsPending) { @@ -39,7 +39,7 @@ public GarnetStatus GET_WithPending(ref SpanByte key, ref RawStringInp } public bool GET_CompletePending((GarnetStatus, SpanByteAndMemory)[] outputArr, bool wait, ref TContext context) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { Debug.Assert(outputArr != null); @@ -62,8 +62,8 @@ public bool GET_CompletePending((GarnetStatus, SpanByteAndMemory)[] ou return ret; } - public bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait, ref TContext context) - where TContext : ITsavoriteContext + public bool GET_CompletePending(out CompletedOutputIterator completedOutputs, bool wait, ref TContext context) + where TContext : ITsavoriteContext { latencyMetrics?.Start(LatencyMetricsType.PENDING_LAT); var ret = context.CompletePendingWithOutputs(out completedOutputs, wait); @@ -71,10 +71,10 @@ public bool GET_CompletePending(out CompletedOutputIterator(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus RMW_MainStore(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.RMW(ref key, ref input, ref output); + var status = context.RMW(key, ref input, ref output); if (status.IsPending) CompletePendingForSession(ref status, ref output, ref context); @@ -85,10 +85,10 @@ public GarnetStatus RMW_MainStore(ref SpanByte key, ref RawStringInput return GarnetStatus.NOTFOUND; } - public GarnetStatus Read_MainStore(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus Read_MainStore(ReadOnlySpan key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.Read(ref key, ref input, ref output); + var status = context.Read(key, ref input, ref output); if (status.IsPending) CompletePendingForSession(ref status, ref output, ref context); diff --git a/libs/server/Storage/Session/MainStore/BitmapOps.cs b/libs/server/Storage/Session/MainStore/BitmapOps.cs index a3b71f43682..2d9c3b161dd 100644 --- a/libs/server/Storage/Session/MainStore/BitmapOps.cs +++ b/libs/server/Storage/Session/MainStore/BitmapOps.cs @@ -4,20 +4,19 @@ using System; using System.Collections.Generic; using System.Diagnostics; -using System.Runtime.CompilerServices; using System.Text; using Garnet.common; using Tsavorite.core; namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { - public unsafe GarnetStatus StringSetBit(ArgSlice key, ArgSlice offset, bool bit, out bool previous, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringSetBit(PinnedSpanByte key, PinnedSpanByte offset, bool bit, out bool previous, ref TContext context) + where TContext : ITsavoriteContext { previous = false; @@ -26,22 +25,20 @@ public unsafe GarnetStatus StringSetBit(ArgSlice key, ArgSlice offset, var setValBytes = stackalloc byte[1]; setValBytes[0] = (byte)(bit ? '1' : '0'); - var setValSlice = new ArgSlice(setValBytes, 1); + var setValSlice = PinnedSpanByte.FromPinnedPointer(setValBytes, 1); parseState.InitializeWithArguments(offset, setValSlice); - var input = new RawStringInput(RespCommand.SETBIT, ref parseState, - arg1: ParseUtils.ReadLong(ref offset)); + var input = new RawStringInput(RespCommand.SETBIT, ref parseState, arg1: ParseUtils.ReadLong(offset)); SpanByteAndMemory output = new(null); - var keySp = key.SpanByte; - RMW_MainStore(ref keySp, ref input, ref output, ref context); + RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); return GarnetStatus.OK; } - public unsafe GarnetStatus StringGetBit(ArgSlice key, ArgSlice offset, out bool bValue, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringGetBit(PinnedSpanByte key, PinnedSpanByte offset, out bool bValue, ref TContext context) + where TContext : ITsavoriteContext { bValue = false; @@ -50,16 +47,14 @@ public unsafe GarnetStatus StringGetBit(ArgSlice key, ArgSlice offset, parseState.InitializeWithArgument(offset); - var input = new RawStringInput(RespCommand.GETBIT, ref parseState, - arg1: ParseUtils.ReadLong(ref offset)); + var input = new RawStringInput(RespCommand.GETBIT, ref parseState, arg1: ParseUtils.ReadLong(offset)); SpanByteAndMemory output = new(null); - var keySp = key.SpanByte; - var status = Read_MainStore(ref keySp, ref input, ref output, ref context); + var status = Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); if (status == GarnetStatus.OK && !output.IsSpanByte) { - fixed (byte* outputPtr = output.Memory.Memory.Span) + fixed (byte* outputPtr = output.MemorySpan) { var refPtr = outputPtr; if (*refPtr == ':') @@ -96,11 +91,11 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp txnManager.SaveKeyEntryToLock(keys[0], false, LockType.Exclusive); for (var i = 1; i < keys.Length; i++) txnManager.SaveKeyEntryToLock(keys[i], false, LockType.Shared); - txnManager.Run(true); + _ = txnManager.Run(true); } // Perform under unsafe epoch control for pointer safety. - var uc = txnManager.LockableUnsafeContext; + var uc = txnManager.TransactionalUnsafeContext; try { @@ -113,7 +108,7 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp { var srcKey = keys[i]; //Read srcKey - var outputBitmap = new SpanByteAndMemory(output, 12); + var outputBitmap = SpanByteAndMemory.FromPinnedPointer(output, 12); status = ReadWithUnsafeContext(srcKey, ref input, ref outputBitmap, localHeadAddress, out bool epochChanged, ref uc); if (epochChanged) { @@ -162,11 +157,11 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp if (maxBitmapLen > 0) { - var dstKey = keys[0].SpanByte; + var dstKey = keys[0]; var valPtr = dstBitmapPtr; valPtr -= sizeof(int); *(int*)valPtr = maxBitmapLen; - status = SET(ref dstKey, ref Unsafe.AsRef(valPtr), ref uc); + status = SET(dstKey, PinnedSpanByte.FromLengthPrefixedPinnedPointer(valPtr), ref uc); } } else @@ -187,13 +182,13 @@ public unsafe GarnetStatus StringBitOperation(ref RawStringInput input, BitmapOp return status; } - public GarnetStatus StringBitOperation(BitmapOperation bitOp, ArgSlice destinationKey, ArgSlice[] keys, out long result) + public GarnetStatus StringBitOperation(BitmapOperation bitOp, PinnedSpanByte destinationKey, PinnedSpanByte[] keys, out long result) { result = 0; if (destinationKey.Length == 0) return GarnetStatus.OK; - var args = new ArgSlice[keys.Length + 1]; + var args = new PinnedSpanByte[keys.Length + 1]; args[0] = destinationKey; keys.CopyTo(args, 1); @@ -204,8 +199,8 @@ public GarnetStatus StringBitOperation(BitmapOperation bitOp, ArgSlice destinati return StringBitOperation(ref input, bitOp, out result); } - public unsafe GarnetStatus StringBitCount(ArgSlice key, long start, long end, bool useBitInterval, out long result, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringBitCount(PinnedSpanByte key, long start, long end, bool useBitInterval, out long result, ref TContext context) + where TContext : ITsavoriteContext { result = 0; @@ -229,19 +224,19 @@ public unsafe GarnetStatus StringBitCount(ArgSlice key, long start, lo // Use bit interval var useBitIntervalSpan = paramsSpan.Slice(paramsSpanOffset, 1); (useBitInterval ? "1"u8 : "0"u8).CopyTo(useBitIntervalSpan); - var useBitIntervalSlice = ArgSlice.FromPinnedSpan(useBitIntervalSpan); + var useBitIntervalSlice = PinnedSpanByte.FromPinnedSpan(useBitIntervalSpan); paramsSpanOffset += 1; // Start var startSpan = paramsSpan.Slice(paramsSpanOffset, startLength); - NumUtils.WriteInt64(start, startSpan); - var startSlice = ArgSlice.FromPinnedSpan(startSpan); + _ = NumUtils.WriteInt64(start, startSpan); + var startSlice = PinnedSpanByte.FromPinnedSpan(startSpan); paramsSpanOffset += startLength; // End var endSpan = paramsSpan.Slice(paramsSpanOffset, endLength); - NumUtils.WriteInt64(end, endSpan); - var endSlice = ArgSlice.FromPinnedSpan(endSpan); + _ = NumUtils.WriteInt64(end, endSpan); + var endSlice = PinnedSpanByte.FromPinnedSpan(endSpan); SpanByteAndMemory output = new(null); @@ -249,20 +244,18 @@ public unsafe GarnetStatus StringBitCount(ArgSlice key, long start, lo var input = new RawStringInput(RespCommand.BITCOUNT, ref parseState); - scratchBufferManager.RewindScratchBuffer(ref paramsSlice); + _ = scratchBufferManager.RewindScratchBuffer(paramsSlice); - var keySp = key.SpanByte; - - var status = Read_MainStore(ref keySp, ref input, ref output, ref context); + var status = Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); if (status == GarnetStatus.OK) { if (!output.IsSpanByte) { - fixed (byte* outputPtr = output.Memory.Memory.Span) + fixed (byte* outputPtr = output.MemorySpan) { var refPtr = outputPtr; - RespReadUtils.TryReadInt64(out result, ref refPtr, refPtr + sizeof(long)); + _ = RespReadUtils.TryReadInt64(out result, ref refPtr, refPtr + sizeof(long)); } output.Memory.Dispose(); } @@ -271,13 +264,12 @@ public unsafe GarnetStatus StringBitCount(ArgSlice key, long start, lo return status; } - public unsafe GarnetStatus StringBitField(ArgSlice key, List commandArguments, out List result, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringBitField(PinnedSpanByte key, List commandArguments, out List result, ref TContext context) + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.BITFIELD); - result = new(); - var keySp = key.SpanByte; + result = []; for (var i = 0; i < commandArguments.Count; i++) { @@ -308,38 +300,38 @@ public unsafe GarnetStatus StringBitField(ArgSlice key, List(ArgSlice key, List(ArgSlice key, List(ArgSlice key, List(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - => RMW_MainStore(ref key, ref input, ref output, ref context); + public GarnetStatus StringSetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext + => RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); - public GarnetStatus StringGetBit(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - => Read_MainStore(ref key, ref input, ref output, ref context); + public GarnetStatus StringGetBit(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext + => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); - public unsafe GarnetStatus StringBitCount(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - => Read_MainStore(ref key, ref input, ref output, ref context); + public unsafe GarnetStatus StringBitCount(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext + => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); - public unsafe GarnetStatus StringBitPosition(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - => Read_MainStore(ref key, ref input, ref output, ref context); + public unsafe GarnetStatus StringBitPosition(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext + => Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); - public unsafe GarnetStatus StringBitField(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringBitField(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { GarnetStatus status; if (secondaryCommand == RespCommand.GET) - status = Read_MainStore(ref key, ref input, ref output, ref context); + status = Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); else - status = RMW_MainStore(ref key, ref input, ref output, ref context); + status = RMW_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); return status; } - public unsafe GarnetStatus StringBitFieldReadOnly(ref SpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus StringBitFieldReadOnly(PinnedSpanByte key, ref RawStringInput input, RespCommand secondaryCommand, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - GarnetStatus status = GarnetStatus.NOTFOUND; + var status = GarnetStatus.NOTFOUND; if (secondaryCommand == RespCommand.GET) - status = Read_MainStore(ref key, ref input, ref output, ref context); + status = Read_MainStore(key.ReadOnlySpan, ref input, ref output, ref context); return status; } diff --git a/libs/server/Storage/Session/MainStore/CompletePending.cs b/libs/server/Storage/Session/MainStore/CompletePending.cs index 94a69ecb413..59625f713cf 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 MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = 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 d649c802a32..be0d00eb634 100644 --- a/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs +++ b/libs/server/Storage/Session/MainStore/HyperLogLogOps.cs @@ -7,16 +7,16 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; 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(ArgSlice key, string[] elements, out bool updated, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus HyperLogLogAdd(PinnedSpanByte key, string[] elements, out bool updated, ref TContext context) + where TContext : ITsavoriteContext { updated = false; @@ -32,11 +32,10 @@ public unsafe GarnetStatus HyperLogLogAdd(ArgSlice key, string[] eleme var elementSlice = scratchBufferManager.CreateArgSlice(element); parseState.SetArgument(0, elementSlice); - var o = new SpanByteAndMemory(output, 1); - var sbKey = key.SpanByte; - RMW_MainStore(ref sbKey, ref input, ref o, ref context); + var o = SpanByteAndMemory.FromPinnedPointer(output, 1); + _ = RMW_MainStore(key.ReadOnlySpan, ref input, ref o, ref context); - scratchBufferManager.RewindScratchBuffer(ref elementSlice); + _ = scratchBufferManager.RewindScratchBuffer(elementSlice); //Invalid HLL Type if (*output == (byte)0xFF) @@ -60,12 +59,12 @@ public unsafe GarnetStatus HyperLogLogAdd(ArgSlice key, string[] eleme /// /// /// - public GarnetStatus HyperLogLogAdd(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - => RMW_MainStore(ref key, ref input, ref output, ref context); + public GarnetStatus HyperLogLogAdd(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + 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 + public unsafe GarnetStatus HyperLogLogLength(Span keys, out long count, ref TContext context) + where TContext : ITsavoriteContext { parseState.Initialize(keys.Length); for (var i = 0; i < keys.Length; i++) @@ -88,7 +87,7 @@ public unsafe GarnetStatus HyperLogLogLength(Span keys, out /// /// 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; @@ -109,10 +108,10 @@ public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, var currSrcKey = input.parseState.GetArgSliceByRef(i); txnManager.SaveKeyEntryToLock(currSrcKey, false, LockType.Shared); } - txnManager.Run(true); + _ = txnManager.Run(true); } - var currLockableContext = txnManager.LockableContext; + var currTransactionalContext = txnManager.TransactionalContext; try { @@ -122,17 +121,17 @@ public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, sectorAlignedMemoryPoolAlignment); var srcReadBuffer = sectorAlignedMemoryHll1.GetValidPointer(); var dstReadBuffer = sectorAlignedMemoryHll2.GetValidPointer(); - var dstMergeBuffer = new SpanByteAndMemory(srcReadBuffer, hllBufferSize); - var srcMergeBuffer = new SpanByteAndMemory(dstReadBuffer, hllBufferSize); + var dstMergeBuffer = SpanByteAndMemory.FromPinnedPointer(srcReadBuffer, hllBufferSize); + var srcMergeBuffer = SpanByteAndMemory.FromPinnedPointer(dstReadBuffer, hllBufferSize); var isFirst = false; for (var i = 0; i < input.parseState.Count; i++) { var currInput = new RawStringInput(RespCommand.PFCOUNT); - var srcKey = input.parseState.GetArgSliceByRef(i).SpanByte; + var srcKey = input.parseState.GetArgSliceByRef(i); - var status = GET(ref srcKey, ref currInput, ref srcMergeBuffer, ref currLockableContext); + var status = GET(srcKey, ref currInput, ref srcMergeBuffer, ref currTransactionalContext); // Handle case merging source key does not exist if (status == GarnetStatus.NOTFOUND) continue; @@ -159,7 +158,7 @@ public unsafe GarnetStatus HyperLogLogLength(ref RawStringInput input, continue; } - HyperLogLog.DefaultHLL.TryMerge(srcHLL, dstHLL, sbDstHLL.Length); + _ = HyperLogLog.DefaultHLL.TryMerge(srcHLL, dstHLL, sbDstHLL.Length); if (i == input.parseState.Count - 1) { @@ -202,17 +201,17 @@ public unsafe GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool e var currSrcKey = input.parseState.GetArgSliceByRef(i); txnManager.SaveKeyEntryToLock(currSrcKey, false, LockType.Shared); } - txnManager.Run(true); + _ = txnManager.Run(true); } - var currLockableContext = txnManager.LockableContext; + var currTransactionalContext = txnManager.TransactionalContext; try { sectorAlignedMemoryHll1 ??= new SectorAlignedMemory(hllBufferSize + sectorAlignedMemoryPoolAlignment, sectorAlignedMemoryPoolAlignment); var readBuffer = sectorAlignedMemoryHll1.GetValidPointer(); - var dstKey = input.parseState.GetArgSliceByRef(0).SpanByte; + var dstKey = input.parseState.GetArgSliceByRef(0); for (var i = 1; i < input.parseState.Count; i++) { @@ -220,10 +219,10 @@ public unsafe GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool e var currInput = new RawStringInput(RespCommand.PFMERGE); - var mergeBuffer = new SpanByteAndMemory(readBuffer, hllBufferSize); - var srcKey = input.parseState.GetArgSliceByRef(i).SpanByte; + var mergeBuffer = SpanByteAndMemory.FromPinnedPointer(readBuffer, hllBufferSize); + var srcKey = input.parseState.GetArgSliceByRef(i); - var status = GET(ref srcKey, ref currInput, ref mergeBuffer, ref currLockableContext); + var status = GET(srcKey, ref currInput, ref mergeBuffer, ref currTransactionalContext); // Handle case merging source key does not exist if (status == GarnetStatus.NOTFOUND) continue; @@ -238,12 +237,12 @@ public unsafe GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool e #region mergeToDst - var mergeSlice = new ArgSlice(ref mergeBuffer.SpanByte); + var mergeSlice = mergeBuffer.SpanByte; parseState.InitializeWithArgument(mergeSlice); currInput.parseState = parseState; - SET_Conditional(ref dstKey, ref currInput, ref mergeBuffer, ref currLockableContext); + SET_Conditional(dstKey, ref currInput, ref mergeBuffer, ref currTransactionalContext); #endregion } diff --git a/libs/server/Storage/Session/MainStore/MainStoreOps.cs b/libs/server/Storage/Session/MainStore/MainStoreOps.cs index f323911bc54..8bc92c4c02b 100644 --- a/libs/server/Storage/Session/MainStore/MainStoreOps.cs +++ b/libs/server/Storage/Session/MainStore/MainStoreOps.cs @@ -10,19 +10,19 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { - public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus GET(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { long ctx = default; - var status = context.Read(ref key, ref input, ref output, ctx); + var status = context.Read(key.ReadOnlySpan, ref input, ref output, ctx); if (status.IsPending) { @@ -43,14 +43,11 @@ public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, re } } - public unsafe GarnetStatus ReadWithUnsafeContext(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output, long localHeadAddress, out bool epochChanged, ref TContext context) - where TContext : ITsavoriteContext, IUnsafeContext + public unsafe GarnetStatus ReadWithUnsafeContext(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long localHeadAddress, out bool epochChanged, ref TContext context) + where TContext : ITsavoriteContext, IUnsafeContext { - var _key = key.SpanByte; - long ctx = default; - epochChanged = false; - var status = context.Read(ref _key, ref Unsafe.AsRef(in input), ref output, ctx); + var status = context.Read(key.ReadOnlySpan, ref Unsafe.AsRef(in input), ref output, userContext: default); if (status.IsPending) { @@ -79,21 +76,20 @@ public unsafe GarnetStatus ReadWithUnsafeContext(ArgSlice key, ref Raw return GarnetStatus.OK; } - public unsafe GarnetStatus GET(ArgSlice key, out ArgSlice value, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus GET(PinnedSpanByte key, out PinnedSpanByte value, ref TContext context) + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GET); value = default; - var _key = key.SpanByte; - var _output = new SpanByteAndMemory { SpanByte = scratchBufferManager.ViewRemainingArgSlice().SpanByte }; + var _output = new SpanByteAndMemory { SpanByte = scratchBufferManager.ViewRemainingArgSlice()}; - var ret = GET(ref _key, ref input, ref _output, ref context); + var ret = GET(key, ref input, ref _output, ref context); if (ret == GarnetStatus.OK) { if (!_output.IsSpanByte) { - value = scratchBufferManager.FormatScratch(0, _output.AsReadOnlySpan()); + value = scratchBufferManager.FormatScratch(0, _output.ReadOnlySpan); _output.Memory.Dispose(); } else @@ -104,24 +100,24 @@ public unsafe GarnetStatus GET(ArgSlice key, out ArgSlice value, ref T return ret; } - public unsafe GarnetStatus GET(ArgSlice key, out MemoryResult value, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus GET(PinnedSpanByte key, out MemoryResult value, ref TContext context) + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GET); - var _key = key.SpanByte; var _output = new SpanByteAndMemory(); - var ret = GET(ref _key, ref input, ref _output, ref context); + var ret = GET(key, ref input, ref _output, ref context); value = new MemoryResult(_output.Memory, _output.Length); return ret; } - public GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus GET(PinnedSpanByte key, out GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { - long ctx = default; - var status = objectContext.Read(key, out output, ctx); + ObjectInput input = default; + output = default; + var status = objectContext.Read(key.ReadOnlySpan, ref input, ref output, userContext: default); if (status.IsPending) { @@ -142,10 +138,10 @@ public GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput } } - public unsafe GarnetStatus GETEX(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus GETEX(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.RMW(ref key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { @@ -173,26 +169,12 @@ public unsafe GarnetStatus GETEX(ref SpanByte key, ref RawStringInput /// Span to allocate the output of the operation /// Basic Context of the store /// Operation status - public unsafe GarnetStatus GETDEL(ArgSlice key, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext - { - var _key = key.SpanByte; - return GETDEL(ref _key, ref output, ref context); - } - - /// - /// GETDEL command - Gets the value corresponding to the given key and deletes the key. - /// - /// The key to get the value for. - /// Span to allocate the output of the operation - /// Basic Context of the store - /// Operation status - public unsafe GarnetStatus GETDEL(ref SpanByte key, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { var input = new RawStringInput(RespCommand.GETDEL); - var status = context.RMW(ref key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); Debug.Assert(output.IsSpanByte); if (status.IsPending) @@ -201,10 +183,10 @@ public unsafe GarnetStatus GETDEL(ref SpanByte key, ref SpanByteAndMem return status.Found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } - public unsafe GarnetStatus GETRANGE(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.Read(ref key, ref input, ref output); + var status = context.Read(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { @@ -238,16 +220,16 @@ public unsafe GarnetStatus GETRANGE(ref SpanByte key, ref RawStringInp /// Object Context of the store /// when true the command to execute is PTTL. /// - public unsafe GarnetStatus TTL(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output, ref TContext context, ref TObjectContext objectContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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 { var cmd = milliseconds ? RespCommand.PTTL : RespCommand.TTL; var input = new RawStringInput(cmd); if (storeType == StoreType.Main || storeType == StoreType.All) { - var status = context.Read(ref key, ref input, ref output); + var status = context.Read(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { @@ -256,7 +238,8 @@ public unsafe GarnetStatus TTL(ref SpanByte key, Store StopPendingMetrics(); } - if (status.Found) return GarnetStatus.OK; + if (status.Found) + return GarnetStatus.OK; } if ((storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) @@ -264,9 +247,8 @@ public unsafe GarnetStatus TTL(ref SpanByte key, Store var header = new RespInputHeader(milliseconds ? GarnetObjectType.PTtl : GarnetObjectType.Ttl); var objInput = new ObjectInput(header); - var keyBA = key.ToByteArray(); var objO = new GarnetObjectStoreOutput(output); - var status = objectContext.Read(ref keyBA, ref objInput, ref objO); + var status = objectContext.Read(key.ReadOnlySpan, ref objInput, ref objO); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref objO, ref objectContext); @@ -292,15 +274,15 @@ public unsafe GarnetStatus TTL(ref SpanByte key, 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(ref SpanByte key, StoreType storeType, ref SpanByteAndMemory output, ref TContext context, ref TObjectContext objectContext, bool milliseconds = false) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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(ref key, ref input, ref output); + var status = context.Read(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { @@ -318,9 +300,8 @@ public unsafe GarnetStatus EXPIRETIME(ref SpanByte key var header = new RespInputHeader(type); var input = new ObjectInput(header); - var keyBA = key.ToByteArray(); var objO = new GarnetObjectStoreOutput(output); - var status = objectContext.Read(ref keyBA, ref input, ref objO); + var status = objectContext.Read(key.ReadOnlySpan, ref input, ref objO); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref objO, ref objectContext); @@ -334,28 +315,28 @@ public unsafe GarnetStatus EXPIRETIME(ref SpanByte key return GarnetStatus.NOTFOUND; } - public GarnetStatus SET(ref SpanByte key, ref SpanByte value, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value, ref TContext context) + where TContext : ITsavoriteContext { - context.Upsert(ref key, ref value); + context.Upsert(key.ReadOnlySpan, value.ReadOnlySpan); return GarnetStatus.OK; } - public GarnetStatus SET(ref SpanByte key, ref RawStringInput input, ref SpanByte value, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus SET(PinnedSpanByte key, ref RawStringInput input, PinnedSpanByte value, ref TContext context) + where TContext : ITsavoriteContext { var output = new SpanByteAndMemory(); - context.Upsert(ref key, ref input, ref value, ref output); + context.Upsert(key.ReadOnlySpan, ref input, value.ReadOnlySpan, ref output); return GarnetStatus.OK; } - public unsafe GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref TContext context) + where TContext : ITsavoriteContext { byte* pbOutput = stackalloc byte[8]; - var o = new SpanByteAndMemory(pbOutput, 8); + var o = SpanByteAndMemory.FromPinnedPointer(pbOutput, 8); - var status = context.RMW(ref key, ref input, ref o); + var status = context.RMW(key.ReadOnlySpan, ref input, ref o); if (status.IsPending) { @@ -377,19 +358,19 @@ public unsafe GarnetStatus SET_Conditional(ref SpanByte key, ref RawSt } - public unsafe GarnetStatus DEL_Conditional(ref SpanByte key, ref RawStringInput input, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus DEL_Conditional(PinnedSpanByte key, ref RawStringInput input, ref TContext context) + where TContext : ITsavoriteContext { Debug.Assert(input.header.cmd == RespCommand.DELIFGREATER); - byte* pbOutput = stackalloc byte[8]; - var o = new SpanByteAndMemory(pbOutput, 8); - var status = context.RMW(ref key, ref input, ref o); + Span outputSpan = stackalloc byte[8]; + var output = SpanByteAndMemory.FromPinnedSpan(outputSpan); + var status = context.RMW(key, ref input, ref output); if (status.IsPending) { StartPendingMetrics(); - CompletePendingForSession(ref status, ref o, ref context); + CompletePendingForSession(ref status, ref output, ref context); StopPendingMetrics(); } @@ -408,10 +389,10 @@ public unsafe GarnetStatus DEL_Conditional(ref SpanByte key, ref RawSt } } - public unsafe GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus SET_Conditional(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.RMW(ref key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { @@ -433,7 +414,7 @@ public unsafe GarnetStatus SET_Conditional(ref SpanByte key, ref RawSt } internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref TContext ctx) - where TContext : ITsavoriteContext + where TContext : ITsavoriteContext { var error = false; var count = input.parseState.Count; @@ -451,8 +432,8 @@ internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref T txnManager.Run(true); } - var context = txnManager.LockableContext; - var objContext = txnManager.ObjectStoreLockableContext; + var context = txnManager.TransactionalContext; + var objContext = txnManager.ObjectStoreTransactionalContext; try { @@ -483,46 +464,46 @@ internal GarnetStatus MSET_Conditional(ref RawStringInput input, ref T return error ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } - public GarnetStatus SET(ArgSlice key, ArgSlice value, ref TContext context) - where TContext : ITsavoriteContext - { - var _key = key.SpanByte; - var _value = value.SpanByte; - return SET(ref _key, ref _value, ref context); - } - - public GarnetStatus SET(byte[] key, IGarnetObject value, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SET(PinnedSpanByte key, IGarnetObject value, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { - objectContext.Upsert(key, value); + objectContext.Upsert(key.ReadOnlySpan, value); return GarnetStatus.OK; } - public GarnetStatus SET(ArgSlice key, Memory value, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus SET(PinnedSpanByte key, Memory value, ref TContext context) // TODO are memory overloads needed? + where TContext : ITsavoriteContext { - var _key = key.SpanByte; unsafe { fixed (byte* ptr = value.Span) - { - var _value = SpanByte.FromPinnedPointer(ptr, value.Length); - return SET(ref _key, ref _value, ref context); - } + context.Upsert(key.ReadOnlySpan, new ReadOnlySpan(ptr, value.Length)); } + return GarnetStatus.OK; + } + + public GarnetStatus SET(ref TSourceLogRecord srcLogRecord, StoreType storeType, ref TContext context, ref TObjectContext objectContext) + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext + where TSourceLogRecord : ISourceLogRecord + { + if (storeType == StoreType.Main) + context.Upsert(ref srcLogRecord); + else + objectContext.Upsert(ref srcLogRecord); + return GarnetStatus.OK; } - public unsafe GarnetStatus SETEX(ArgSlice key, ArgSlice value, ArgSlice expiryMs, ref TContext context) - where TContext : ITsavoriteContext - => SETEX(key, value, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ptr)), ref context); + public unsafe GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, PinnedSpanByte expiryMs, ref TContext context) + where TContext : ITsavoriteContext + => SETEX(key, value, TimeSpan.FromMilliseconds(NumUtils.ReadInt64(expiryMs.Length, expiryMs.ToPointer())), ref context); - public GarnetStatus SETEX(ArgSlice key, ArgSlice value, TimeSpan expiry, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus SETEX(PinnedSpanByte key, PinnedSpanByte value, TimeSpan expiry, ref TContext context) + where TContext : ITsavoriteContext { - var _key = key.SpanByte; - var valueSB = scratchBufferManager.FormatScratch(sizeof(long), value).SpanByte; - valueSB.ExtraMetadata = DateTimeOffset.UtcNow.Ticks + expiry.Ticks; - return SET(ref _key, ref valueSB, ref context); + parseState.InitializeWithArgument(value); + var input = new RawStringInput(RespCommand.APPEND, ref parseState, arg1: DateTimeOffset.UtcNow.Ticks + expiry.Ticks); + return SET(key, ref input, value, ref context); } /// @@ -534,17 +515,15 @@ public GarnetStatus SETEX(ArgSlice key, ArgSlice value, TimeSpan expir /// Length of updated value /// Store context /// Operation status - public unsafe GarnetStatus APPEND(ArgSlice key, ArgSlice value, ref ArgSlice output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus APPEND(PinnedSpanByte key, PinnedSpanByte value, ref PinnedSpanByte output, ref TContext context) + where TContext : ITsavoriteContext { - var _key = key.SpanByte; - var _output = new SpanByteAndMemory(output.SpanByte); + var _output = new SpanByteAndMemory(output); parseState.InitializeWithArgument(value); - var input = new RawStringInput(RespCommand.APPEND, ref parseState); - return APPEND(ref _key, ref input, ref _output, ref context); + return APPEND(key, ref input, ref _output, ref context); } /// @@ -556,10 +535,10 @@ public unsafe GarnetStatus APPEND(ArgSlice key, ArgSlice value, ref Ar /// Length of updated value /// Store context /// Operation status - public unsafe GarnetStatus APPEND(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus APPEND(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, ref TContext context) + where TContext : ITsavoriteContext { - var status = context.RMW(ref key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) { StartPendingMetrics(); @@ -572,70 +551,30 @@ public unsafe GarnetStatus APPEND(ref SpanByte key, ref RawStringInput return GarnetStatus.OK; } - public GarnetStatus DELETE(ArgSlice key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - var _key = key.SpanByte; - return DELETE(ref _key, storeType, ref context, ref objectContext); - } - - public GarnetStatus DELETE(ref SpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var found = false; if (storeType == StoreType.Main || storeType == StoreType.All) { - var status = context.Delete(ref key); + 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 keyBA = key.ToByteArray(); - var status = objectContext.Delete(ref keyBA); + var status = objectContext.Delete(key.ReadOnlySpan); Debug.Assert(!status.IsPending); if (status.Found) found = true; } return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } - public GarnetStatus DELETE(byte[] key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext - { - bool found = false; - - if ((storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) - { - var status = objectContext.Delete(key); - Debug.Assert(!status.IsPending); - if (status.Found) found = true; - } - - if (!found && (storeType == StoreType.Main || storeType == StoreType.All)) - { - unsafe - { - fixed (byte* ptr = key) - { - var keySB = SpanByte.FromPinnedPointer(ptr, key.Length); - var status = context.Delete(ref keySB); - Debug.Assert(!status.IsPending); - if (status.Found) found = true; - } - } - } - return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; - } - - public unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, StoreType storeType, bool withEtag) - { - return RENAME(oldKeySlice, newKeySlice, storeType, false, out _, withEtag); - } + public unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte newKeySlice, StoreType storeType, bool withEtag) + => RENAME(oldKeySlice, newKeySlice, storeType, false, out _, withEtag); /// /// Renames key to newkey if newkey does not yet exist. It returns an error when key does not exist. @@ -644,12 +583,10 @@ public unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, St /// The new key name. /// The type of store to perform the operation on. /// - public unsafe GarnetStatus RENAMENX(ArgSlice oldKeySlice, ArgSlice newKeySlice, StoreType storeType, out int result, bool withEtag) - { - return RENAME(oldKeySlice, newKeySlice, storeType, true, out result, withEtag); - } + public unsafe GarnetStatus RENAMENX(PinnedSpanByte oldKeySlice, PinnedSpanByte newKeySlice, StoreType storeType, out int result, bool withEtag) + => RENAME(oldKeySlice, newKeySlice, storeType, true, out result, withEtag); - private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, StoreType storeType, bool isNX, out int result, bool withEtag) + private unsafe GarnetStatus RENAME(PinnedSpanByte oldKeySlice, PinnedSpanByte newKeySlice, StoreType storeType, bool isNX, out int result, bool withEtag) { RawStringInput input = default; var returnStatus = GarnetStatus.NOTFOUND; @@ -668,21 +605,24 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S createTransaction = true; txnManager.SaveKeyEntryToLock(oldKeySlice, false, LockType.Exclusive); txnManager.SaveKeyEntryToLock(newKeySlice, false, LockType.Exclusive); - txnManager.Run(true); + _ = txnManager.Run(true); } - var context = txnManager.LockableContext; - var objectContext = txnManager.ObjectStoreLockableContext; - var oldKey = oldKeySlice.SpanByte; + var context = txnManager.TransactionalContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; + var oldKey = oldKeySlice; + + // TODO: This needs to be converted to a form of GET that returns all information in the (Disk)LogRecord, perhaps serializing it to the output, and then + // inserts with that record. if (storeType == StoreType.Main || storeType == StoreType.All) { try { - var newKey = newKeySlice.SpanByte; + var newKey = newKeySlice; var o = new SpanByteAndMemory(); - var status = GET(ref oldKey, ref input, ref o, ref context); + var status = GET(oldKey, ref input, ref o, ref context); if (status == GarnetStatus.OK) { @@ -690,19 +630,19 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S var memoryHandle = o.Memory.Memory.Pin(); var ptrVal = (byte*)memoryHandle.Pointer; - RespReadUtils.TryReadUnsignedLengthHeader(out var headerLength, ref ptrVal, ptrVal + o.Length); + _ = RespReadUtils.TryReadUnsignedLengthHeader(out var headerLength, ref ptrVal, ptrVal + o.Length); // Find expiration time of the old key var expireSpan = new SpanByteAndMemory(); - var ttlStatus = TTL(ref oldKey, storeType, ref expireSpan, ref context, ref objectContext, true); + var ttlStatus = TTL(oldKey, storeType, ref expireSpan, ref context, ref objectContext, true); if (ttlStatus == GarnetStatus.OK && !expireSpan.IsSpanByte) { - var newValSlice = new ArgSlice(ptrVal, headerLength); + var newValSlice = PinnedSpanByte.FromPinnedPointer(ptrVal, headerLength); using var expireMemoryHandle = expireSpan.Memory.Memory.Pin(); var expirePtrVal = (byte*)expireMemoryHandle.Pointer; - RespReadUtils.TryReadInt64(out var expireTimeMs, ref expirePtrVal, expirePtrVal + expireSpan.Length, out var _); + _ = RespReadUtils.TryReadInt64(out var expireTimeMs, ref expirePtrVal, expirePtrVal + expireSpan.Length, out var _); input = isNX ? new RawStringInput(RespCommand.SETEXNX) : new RawStringInput(RespCommand.SET); @@ -721,27 +661,22 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S input.arg1 = DateTimeOffset.UtcNow.Ticks + TimeSpan.FromMilliseconds(expireTimeMs).Ticks; if (withEtag) - { - input.header.SetWithEtagFlag(); - } - - var setStatus = SET_Conditional(ref newKey, ref input, ref context); + input.header.SetWithETagFlag(); + var setStatus = SET_Conditional(newKey, ref input, ref context); if (isNX) { // For SET NX `NOTFOUND` means the operation succeeded result = setStatus == GarnetStatus.NOTFOUND ? 1 : 0; returnStatus = GarnetStatus.OK; } + SETEX(newKeySlice, newValSlice, TimeSpan.FromMilliseconds(expireTimeMs), ref context); } } else if (expireTimeMs == -1) // Its possible to have expireTimeMs as 0 (Key expired or will be expired now) or -2 (Key does not exist), in those cases we don't SET the new key { if (!withEtag && !isNX) - { - var value = newValSlice.SpanByte; - SET(ref newKey, ref value, ref context); - } + SET(newKey, newValSlice, ref context); else { // Build parse state @@ -749,11 +684,9 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S input.parseState = parseState; if (withEtag) - { - input.header.SetWithEtagFlag(); - } + input.header.SetWithETagFlag(); - var setStatus = SET_Conditional(ref newKey, ref input, ref context); + var setStatus = SET_Conditional(newKey, ref input, ref context); if (isNX) { @@ -771,13 +704,12 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S // Delete the old key only when SET NX succeeded if (isNX && result == 1) { - DELETE(ref oldKey, StoreType.Main, ref context, ref objectContext); + DELETE(oldKey, StoreType.Main, ref context, ref objectContext); } else if (!isNX) { // Delete the old key - DELETE(ref oldKey, StoreType.Main, ref context, ref objectContext); - + DELETE(oldKey, StoreType.Main, ref context, ref objectContext); returnStatus = GarnetStatus.OK; } } @@ -803,30 +735,28 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S try { - byte[] oldKeyArray = oldKeySlice.ToArray(); - var status = GET(oldKeyArray, out var value, ref objectContext); + var status = GET(oldKeySlice, out var value, ref objectContext); if (status == GarnetStatus.OK) { var valObj = value.GarnetObject; - byte[] newKeyArray = newKeySlice.ToArray(); returnStatus = GarnetStatus.OK; var canSetAndDelete = true; if (isNX) { // Not using EXISTS method to avoid new allocation of Array for key - var getNewStatus = GET(newKeyArray, out _, ref objectContext); + var getNewStatus = GET(newKeySlice, out _, ref objectContext); canSetAndDelete = getNewStatus == GarnetStatus.NOTFOUND; } if (canSetAndDelete) { - // valObj already has expiration time, so no need to write expiration logic here - SET(newKeyArray, valObj, ref objectContext); + // valObj already has expiration time, so no need to write expiration logic here. TODO: No longer true; this is now a LogRecord attribute and must be SETEX'd + SET(newKeySlice, valObj, ref objectContext); // Delete the old key - DELETE(oldKeyArray, StoreType.Object, ref context, ref objectContext); + DELETE(oldKeySlice, StoreType.Object, ref context, ref objectContext); result = 1; } @@ -855,18 +785,17 @@ private unsafe GarnetStatus RENAME(ArgSlice oldKeySlice, ArgSlice newKeySlice, S /// Basic context for the main store. /// Object context for the object store. /// - public GarnetStatus EXISTS(ArgSlice key, StoreType storeType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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 _key = key.SpanByte; - var _output = new SpanByteAndMemory { SpanByte = scratchBufferManager.ViewRemainingArgSlice().SpanByte }; - status = GET(ref _key, ref input, ref _output, ref context); + var _output = new SpanByteAndMemory { SpanByte = scratchBufferManager.ViewRemainingArgSlice() }; + status = GET(key, ref input, ref _output, ref context); if (status == GarnetStatus.OK) { @@ -878,7 +807,7 @@ public GarnetStatus EXISTS(ArgSlice key, StoreType sto if ((storeType == StoreType.Object || storeType == StoreType.All) && !objectStoreBasicContext.IsNull) { - status = GET(key.ToArray(), out _, ref objectContext); + status = GET(key, out _, ref objectContext); } return status; @@ -897,10 +826,10 @@ public GarnetStatus EXISTS(ArgSlice key, StoreType sto /// Basic context for the main store. /// Object context for the object store. /// - public unsafe GarnetStatus EXPIRE(ArgSlice key, ArgSlice 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.ptr)), out timeoutSet, storeType, expireOption, ref context, ref objectStoreContext); + 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. @@ -914,20 +843,19 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, ArgSli /// Basic context for the main store /// Object context for the object store /// - public unsafe GarnetStatus EXPIRE(ArgSlice key, ref RawStringInput input, out bool timeoutSet, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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 { - var rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; - var output = new SpanByteAndMemory(SpanByte.FromPinnedPointer(rmwOutput, ObjectOutputHeader.Size)); + Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; + var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); timeoutSet = false; var found = false; if (storeType == StoreType.Main || storeType == StoreType.All) { - var _key = key.SpanByte; - var status = context.RMW(ref _key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) CompletePendingForSession(ref status, ref output, ref context); @@ -951,8 +879,7 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, ref Ra // Retry on object store var objOutput = new GarnetObjectStoreOutput(output); - var keyBytes = key.ToArray(); - var status = objectStoreContext.RMW(ref keyBytes, ref objInput, ref objOutput); + var status = objectStoreContext.RMW(key.ReadOnlySpan, ref objInput, ref objOutput); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref objOutput, ref objectStoreContext); @@ -983,9 +910,9 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, ref Ra /// 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(ArgSlice 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 + 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); } @@ -1004,9 +931,9 @@ public unsafe GarnetStatus EXPIREAT(ArgSlice key, long /// 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(ArgSlice 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 + 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); @@ -1026,12 +953,12 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, TimeSp /// Object context for the object store /// The current RESP command /// - public unsafe GarnetStatus EXPIRE(ArgSlice key, long expiry, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, RespCommand respCommand) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, long expiry, out bool timeoutSet, StoreType storeType, ExpireOption expireOption, ref TContext context, ref TObjectContext objectStoreContext, RespCommand respCommand) + where TContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { - var rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; - var output = new SpanByteAndMemory(SpanByte.FromPinnedPointer(rmwOutput, ObjectOutputHeader.Size)); + Span rmwOutput = stackalloc byte[ObjectOutputHeader.Size]; + var output = SpanByteAndMemory.FromPinnedSpan(rmwOutput); timeoutSet = false; var found = false; @@ -1047,9 +974,7 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, long e parseState.InitializeWithArgument(expirySlice); var input = new RawStringInput(respCommand, ref parseState, arg1: (byte)expireOption); - - var _key = key.SpanByte; - var status = context.RMW(ref _key, ref input, ref output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref output); if (status.IsPending) CompletePendingForSession(ref status, ref output, ref context); @@ -1074,7 +999,7 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, long e // Retry on object store var objOutput = new GarnetObjectStoreOutput(output); var keyBytes = key.ToArray(); - var status = objectStoreContext.RMW(ref keyBytes, ref objInput, ref objOutput); + var status = objectStoreContext.RMW(key.ReadOnlySpan, ref objInput, ref objOutput); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref objOutput, ref objectStoreContext); @@ -1083,7 +1008,7 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, long e output = objOutput.SpanByteAndMemory; } - scratchBufferManager.RewindScratchBuffer(ref expirySlice); + scratchBufferManager.RewindScratchBuffer(expirySlice); Debug.Assert(output.IsSpanByte); if (found) timeoutSet = ((ObjectOutputHeader*)output.SpanByte.ToPointer())->result1 == 1; @@ -1091,27 +1016,26 @@ public unsafe GarnetStatus EXPIRE(ArgSlice key, long e return found ? GarnetStatus.OK : GarnetStatus.NOTFOUND; } - public unsafe GarnetStatus PERSIST(ArgSlice key, StoreType storeType, ref TContext context, ref TObjectContext objectStoreContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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 = new SpanByteAndMemory(pbOutput, 8); + var o = SpanByteAndMemory.FromPinnedPointer(pbOutput, 8); if (storeType == StoreType.Main || storeType == StoreType.All) { - var _key = key.SpanByte; - var _status = context.RMW(ref _key, ref inputHeader, ref o); + 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.AsReadOnlySpan()[0] == 1) + if (o.SpanByte.ReadOnlySpan[0] == 1) status = GarnetStatus.OK; } @@ -1123,13 +1047,13 @@ public unsafe GarnetStatus PERSIST(ArgSlice key, Store var objO = new GarnetObjectStoreOutput(o); var _key = key.ToArray(); - var _status = objectStoreContext.RMW(ref _key, ref objInput, ref objO); + 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.AsReadOnlySpan().Slice(0, CmdStrings.RESP_RETURN_VAL_1.Length) + if (o.SpanByte.ReadOnlySpan.Slice(0, CmdStrings.RESP_RETURN_VAL_1.Length) .SequenceEqual(CmdStrings.RESP_RETURN_VAL_1)) status = GarnetStatus.OK; } @@ -1147,38 +1071,36 @@ public unsafe GarnetStatus PERSIST(ArgSlice key, Store /// The length of the updated string /// Basic context for the main store /// - public unsafe GarnetStatus SETRANGE(ArgSlice key, ref RawStringInput input, ref ArgSlice output, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus SETRANGE(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output, ref TContext context) + where TContext : ITsavoriteContext { - var sbKey = key.SpanByte; - SpanByteAndMemory sbmOut = new(output.SpanByte); + SpanByteAndMemory sbmOut = new(output); - var status = context.RMW(ref sbKey, ref input, ref sbmOut); + var status = context.RMW(key.ReadOnlySpan, ref input, ref sbmOut); if (status.IsPending) CompletePendingForSession(ref status, ref sbmOut, ref context); Debug.Assert(sbmOut.IsSpanByte); - output.length = sbmOut.Length; + output.Length = sbmOut.Length; return GarnetStatus.OK; } - public GarnetStatus Increment(ArgSlice key, ref RawStringInput input, ref ArgSlice output, ref TContext context) - where TContext : ITsavoriteContext + public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output, ref TContext context) + where TContext : ITsavoriteContext { - var _key = key.SpanByte; - SpanByteAndMemory _output = new(output.SpanByte); + SpanByteAndMemory _output = new(output); - var status = context.RMW(ref _key, ref input, ref _output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref _output); if (status.IsPending) CompletePendingForSession(ref status, ref _output, ref context); Debug.Assert(_output.IsSpanByte); - output.length = _output.Length; + output.Length = _output.Length; return GarnetStatus.OK; } - public unsafe GarnetStatus Increment(ArgSlice key, out long output, long increment, ref TContext context) - where TContext : ITsavoriteContext + public unsafe GarnetStatus Increment(PinnedSpanByte key, out long output, long increment, ref TContext context) + where TContext : ITsavoriteContext { var cmd = RespCommand.INCRBY; if (increment < 0) @@ -1192,10 +1114,9 @@ public unsafe GarnetStatus Increment(ArgSlice key, out long output, lo const int outputBufferLength = NumUtils.MaximumFormatInt64Length + 1; var outputBuffer = stackalloc byte[outputBufferLength]; - var _key = key.SpanByte; - var _output = new SpanByteAndMemory(outputBuffer, outputBufferLength); + var _output = SpanByteAndMemory.FromPinnedPointer(outputBuffer, outputBufferLength); - var status = context.RMW(ref _key, ref input, ref _output); + var status = context.RMW(key.ReadOnlySpan, ref input, ref _output); if (status.IsPending) CompletePendingForSession(ref status, ref _output, ref context); @@ -1205,27 +1126,13 @@ public unsafe GarnetStatus Increment(ArgSlice key, out long output, lo return GarnetStatus.OK; } - public void WATCH(ArgSlice key, StoreType type) - { - txnManager.Watch(key, type); - } + public void WATCH(PinnedSpanByte key, StoreType type) => txnManager.Watch(key, type); - public unsafe void WATCH(byte[] key, StoreType type) - { - fixed (byte* ptr = key) - { - WATCH(new ArgSlice(ptr, key.Length), type); - } - } - - public unsafe GarnetStatus SCAN(long cursor, ArgSlice match, long count, ref TContext context) - { - return GarnetStatus.OK; - } + public unsafe GarnetStatus SCAN(long cursor, PinnedSpanByte match, long count, ref TContext context) => GarnetStatus.OK; - public GarnetStatus GetKeyType(ArgSlice key, out string keyType, ref TContext context, ref TObjectContext objectContext) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + public GarnetStatus GetKeyType(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 @@ -1234,7 +1141,7 @@ public GarnetStatus GetKeyType(ArgSlice key, out strin // If key was not found in the main store then it is an object if (status != GarnetStatus.OK && !objectStoreBasicContext.IsNull) { - status = GET(key.ToArray(), out GarnetObjectStoreOutput output, ref objectContext); + status = GET(key, out GarnetObjectStoreOutput output, ref objectContext); if (status == GarnetStatus.OK) { if ((output.GarnetObject as SortedSetObject) != null) @@ -1263,28 +1170,28 @@ public GarnetStatus GetKeyType(ArgSlice key, out strin return status; } - public GarnetStatus MemoryUsageForKey(ArgSlice key, out long memoryUsage, ref TContext context, ref TObjectContext objectContext, int samples = 0) - where TContext : ITsavoriteContext - where TObjectContext : ITsavoriteContext + 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 ArgSlice keyValue, ref context); + var status = GET(key, out PinnedSpanByte keyValue, ref context); if (status == GarnetStatus.NOTFOUND) { - status = GET(key.ToArray(), out GarnetObjectStoreOutput objectValue, ref objectContext); + status = GET(key, out GarnetObjectStoreOutput objectValue, ref objectContext); if (status != GarnetStatus.NOTFOUND) { memoryUsage = RecordInfo.GetLength() + (2 * IntPtr.Size) + // Log record length - Utility.RoundUp(key.SpanByte.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead + // Key allocation in heap with overhead - objectValue.GarnetObject.Size; // Value allocation in heap + 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.SpanByte.TotalSize, RecordInfo.GetLength()) + Utility.RoundUp(keyValue.SpanByte.TotalSize, RecordInfo.GetLength()); + memoryUsage = RecordInfo.GetLength() + Utility.RoundUp(key.TotalSize, RecordInfo.GetLength()) + Utility.RoundUp(keyValue.TotalSize, RecordInfo.GetLength()); } return status; @@ -1301,7 +1208,7 @@ public GarnetStatus MemoryUsageForKey(ArgSlice key, ou /// If true, the length of each match is returned. /// The minimum length of a match to be considered. /// The status of the operation. - public unsafe GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) + public unsafe GarnetStatus LCS(PinnedSpanByte key1, PinnedSpanByte key2, ref SpanByteAndMemory output, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) { var createTransaction = false; if (txnManager.state != TxnState.Running) @@ -1312,7 +1219,7 @@ public unsafe GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemo createTransaction = true; } - var context = txnManager.LockableContext; + var context = txnManager.TransactionalContext; try { var status = LCSInternal(key1, key2, ref output, ref context, lenOnly, withIndices, withMatchLen, minMatchLen); @@ -1325,10 +1232,10 @@ public unsafe GarnetStatus LCS(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemo } } - private unsafe GarnetStatus LCSInternal(ArgSlice key1, ArgSlice key2, ref SpanByteAndMemory output, ref TContext context, bool lenOnly = false, bool withIndices = false, bool withMatchLen = false, int minMatchLen = 0) - where TContext : ITsavoriteContext + 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 { - ArgSlice val1, val2; + PinnedSpanByte val1, val2; var status1 = GET(key1, out val1, ref context); var status2 = GET(key2, out val2, ref context); diff --git a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs index 2d98a5ae71e..db6a3a83362 100644 --- a/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs +++ b/libs/server/Storage/Session/ObjectStore/AdvancedOps.cs @@ -6,15 +6,15 @@ namespace Garnet.server { - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { - public GarnetStatus RMW_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus RMW_ObjectStore(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - var status = objectStoreContext.RMW(ref key, ref input, ref output); + var status = objectStoreContext.RMW(key, ref input, ref output); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref output, ref objectStoreContext); @@ -23,17 +23,16 @@ public GarnetStatus RMW_ObjectStore(ref byte[] key, ref ObjectIn { if (output.HasWrongType) return GarnetStatus.WRONGTYPE; - return GarnetStatus.OK; } return GarnetStatus.NOTFOUND; } - public GarnetStatus Read_ObjectStore(ref byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus Read_ObjectStore(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - var status = objectStoreContext.Read(ref key, ref input, ref output); + var status = objectStoreContext.Read(key, ref input, ref output); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref output, ref objectStoreContext); @@ -42,7 +41,6 @@ public GarnetStatus Read_ObjectStore(ref byte[] key, ref ObjectI { if (output.HasWrongType) return GarnetStatus.WRONGTYPE; - return GarnetStatus.OK; } diff --git a/libs/server/Storage/Session/ObjectStore/Common.cs b/libs/server/Storage/Session/ObjectStore/Common.cs index bbad31cd1fd..76d227f7070 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { #region Common ObjectStore Methods - unsafe GarnetStatus RMWObjectStoreOperation(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + unsafe GarnetStatus RMWObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -28,21 +28,21 @@ unsafe GarnetStatus RMWObjectStoreOperation(byte[] key, ref Obje var objStoreOutput = new GarnetObjectStoreOutput(); // Perform RMW on object store - var status = objectStoreContext.RMW(ref key, ref input, ref objStoreOutput); + var status = objectStoreContext.RMW(key, ref input, ref objStoreOutput); output = objStoreOutput.Header; return CompletePendingAndGetGarnetStatus(status, ref objectStoreContext, ref objStoreOutput); } - unsafe GarnetStatus RMWObjectStoreOperation(byte[] key, ArgSlice input, + 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(); - ref var objInput = ref Unsafe.AsRef(input.ptr); + ref var objInput = ref Unsafe.AsRef(input.ToPointer()); return RMWObjectStoreOperation(key, ref objInput, out output, ref objectStoreContext); } @@ -57,14 +57,14 @@ unsafe GarnetStatus RMWObjectStoreOperation(byte[] key, ArgSlice /// /// /// - GarnetStatus RMWObjectStoreOperationWithOutput(byte[] key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + GarnetStatus RMWObjectStoreOperationWithOutput(ReadOnlySpan key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); // Perform RMW on object store - var status = objectStoreContext.RMW(ref key, ref input, ref output); + var status = objectStoreContext.RMW(key, ref input, ref output); return CompletePendingAndGetGarnetStatus(status, ref objectStoreContext, ref output); } @@ -79,14 +79,14 @@ GarnetStatus RMWObjectStoreOperationWithOutput(byte[] key, ref O /// /// /// - GarnetStatus ReadObjectStoreOperationWithOutput(byte[] key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) - where TObjectContext : ITsavoriteContext + GarnetStatus ReadObjectStoreOperationWithOutput(ReadOnlySpan key, ref ObjectInput input, ref TObjectContext objectStoreContext, ref GarnetObjectStoreOutput output) + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); // Perform read on object store - var status = objectStoreContext.Read(ref key, ref input, ref output); + var status = objectStoreContext.Read(key, ref input, ref output); return CompletePendingAndGetGarnetStatus(status, ref objectStoreContext, ref output); } @@ -101,14 +101,14 @@ GarnetStatus ReadObjectStoreOperationWithOutput(byte[] key, ref /// /// /// - unsafe GarnetStatus ReadObjectStoreOperationWithOutput(byte[] key, ArgSlice input, + 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(); - ref var objInput = ref Unsafe.AsRef(input.ptr); + ref var objInput = ref Unsafe.AsRef(input.ToPointer()); return ReadObjectStoreOperationWithOutput(key, ref objInput, ref objectStoreContext, ref output); } @@ -123,8 +123,8 @@ unsafe GarnetStatus ReadObjectStoreOperationWithOutput(byte[] ke /// Limit number for the response /// The list of items for the response /// - public unsafe GarnetStatus ObjectScan(GarnetObjectType objectType, ArgSlice key, long cursor, string match, int count, out ArgSlice[] items, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ObjectScan(GarnetObjectType objectType, PinnedSpanByte key, long cursor, string match, int count, out PinnedSpanByte[] items, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { Debug.Assert(objectType is GarnetObjectType.Hash or GarnetObjectType.Set or GarnetObjectType.SortedSet); @@ -159,30 +159,30 @@ public unsafe GarnetStatus ObjectScan(GarnetObjectType objectTyp var cursorSpan = paramsSpan.Slice(paramsSpanOffset, cursorLength); NumUtils.WriteInt64(cursor, cursorSpan); paramsSpanOffset += cursorLength; - var cursorSlice = ArgSlice.FromPinnedSpan(cursorSpan); + var cursorSlice = PinnedSpanByte.FromPinnedSpan(cursorSpan); // MATCH var matchSpan = paramsSpan.Slice(paramsSpanOffset, CmdStrings.MATCH.Length); CmdStrings.MATCH.CopyTo(matchSpan); paramsSpanOffset += CmdStrings.MATCH.Length; - var matchSlice = ArgSlice.FromPinnedSpan(matchSpan); + var matchSlice = PinnedSpanByte.FromPinnedSpan(matchSpan); // Pattern var patternSpan = paramsSpan.Slice(paramsSpanOffset, matchPattern.Length); - Encoding.ASCII.GetBytes(matchPattern, patternSpan); + _ = Encoding.ASCII.GetBytes(matchPattern, patternSpan); paramsSpanOffset += matchPattern.Length; - var matchPatternSlice = ArgSlice.FromPinnedSpan(patternSpan); + var matchPatternSlice = PinnedSpanByte.FromPinnedSpan(patternSpan); // COUNT var countSpan = paramsSpan.Slice(paramsSpanOffset, CmdStrings.COUNT.Length); CmdStrings.COUNT.CopyTo(countSpan); paramsSpanOffset += CmdStrings.COUNT.Length; - var countSlice = ArgSlice.FromPinnedSpan(countSpan); + var countSlice = PinnedSpanByte.FromPinnedSpan(countSpan); // Value var countValueSpan = paramsSpan.Slice(paramsSpanOffset, countLength); - NumUtils.WriteInt64(count, countValueSpan); - var countValueSlice = ArgSlice.FromPinnedSpan(countValueSpan); + _ = NumUtils.WriteInt64(count, countValueSpan); + var countValueSlice = PinnedSpanByte.FromPinnedSpan(countValueSpan); parseState.InitializeWithArguments(cursorSlice, matchSlice, matchPatternSlice, countSlice, countValueSlice); @@ -205,9 +205,9 @@ public unsafe GarnetStatus ObjectScan(GarnetObjectType objectTyp } var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); - scratchBufferManager.RewindScratchBuffer(ref paramsSlice); + _ = scratchBufferManager.RewindScratchBuffer(paramsSlice); items = default; if (status == GarnetStatus.OK) @@ -224,17 +224,16 @@ public unsafe GarnetStatus ObjectScan(GarnetObjectType objectTyp /// A description of the error, if there is any /// True when the output comes from HSCAN, ZSCAN OR SSCAN command /// - unsafe ArgSlice[] ProcessRespArrayOutput(GarnetObjectStoreOutput output, out string error, bool isScanOutput = false) + unsafe PinnedSpanByte[] ProcessRespArrayOutput(GarnetObjectStoreOutput output, out string error, bool isScanOutput = false) { - ArgSlice[] elements = default; + PinnedSpanByte[] elements = default; error = default; // For reading the elements in the output byte* element = null; - int len = 0; + var len = 0; - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { @@ -267,11 +266,11 @@ unsafe ArgSlice[] ProcessRespArrayOutput(GarnetObjectStoreOutput output, out str return default; // Create the argslice[] - elements = new ArgSlice[isScanOutput ? arraySize + 1 : arraySize]; + elements = new PinnedSpanByte[isScanOutput ? arraySize + 1 : arraySize]; - int i = 0; + var i = 0; if (isScanOutput) - elements[i++] = new ArgSlice(element, len); + elements[i++] = PinnedSpanByte.FromPinnedPointer(element, len); for (; i < elements.Length; i++) { @@ -279,7 +278,7 @@ unsafe ArgSlice[] ProcessRespArrayOutput(GarnetObjectStoreOutput output, out str len = 0; if (RespReadUtils.TryReadPtrWithLengthHeader(ref element, ref len, ref refPtr, outputPtr + outputSpan.Length)) { - elements[i] = new ArgSlice(element, len); + elements[i] = PinnedSpanByte.FromPinnedPointer(element, len); } } } @@ -289,7 +288,7 @@ unsafe ArgSlice[] ProcessRespArrayOutput(GarnetObjectStoreOutput output, out str len = 0; if (!RespReadUtils.TryReadPtrWithLengthHeader(ref result, ref len, ref refPtr, outputPtr + outputSpan.Length)) return default; - elements = [new ArgSlice(result, len)]; + elements = [PinnedSpanByte.FromPinnedPointer(result, len)]; } } } @@ -316,8 +315,7 @@ unsafe int[] ProcessRespIntegerArrayOutput(GarnetObjectStoreOutput output, out s // For reading the elements in the output byte* element = null; - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { @@ -338,7 +336,7 @@ unsafe int[] ProcessRespIntegerArrayOutput(GarnetObjectStoreOutput output, out s // Create the argslice[] elements = new int[arraySize]; - for (int i = 0; i < elements.Length; i++) + for (var i = 0; i < elements.Length; i++) { if (*refPtr != ':') { @@ -385,8 +383,7 @@ unsafe long[] ProcessRespInt64ArrayOutput(GarnetObjectStoreOutput output, out st // For reading the elements in the output byte* element = null; - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { @@ -443,14 +440,13 @@ unsafe long[] ProcessRespInt64ArrayOutput(GarnetObjectStoreOutput output, out st /// /// Processes RESP output as pairs of score and member. /// - unsafe (ArgSlice member, ArgSlice score)[] ProcessRespArrayOutputAsPairs(GarnetObjectStoreOutput output, out string error) + unsafe (PinnedSpanByte member, PinnedSpanByte score)[] ProcessRespArrayOutputAsPairs(GarnetObjectStoreOutput output, out string error) { - (ArgSlice member, ArgSlice score)[] result = default; + (PinnedSpanByte member, PinnedSpanByte score)[] result = default; error = default; byte* element = null; var len = 0; - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { @@ -471,19 +467,19 @@ unsafe long[] ProcessRespInt64ArrayOutput(GarnetObjectStoreOutput output, out st Debug.Assert(arraySize % 2 == 0, "Array elements are expected to be in pairs"); arraySize /= 2; // Halve the array size to hold items as pairs - result = new (ArgSlice member, ArgSlice score)[arraySize]; + result = new (PinnedSpanByte member, PinnedSpanByte score)[arraySize]; for (var i = 0; i < result.Length; i++) { if (!RespReadUtils.TryReadPtrWithLengthHeader(ref element, ref len, ref refPtr, outputPtr + outputSpan.Length)) return default; - result[i].member = new ArgSlice(element, len); + result[i].member = PinnedSpanByte.FromPinnedPointer(element, len); if (!RespReadUtils.TryReadPtrWithLengthHeader(ref element, ref len, ref refPtr, outputPtr + outputSpan.Length)) return default; - result[i].score = new ArgSlice(element, len); + result[i].score = PinnedSpanByte.FromPinnedPointer(element, len); } } } @@ -502,14 +498,13 @@ unsafe long[] ProcessRespInt64ArrayOutput(GarnetObjectStoreOutput output, out st /// /// The RESP format output object /// - unsafe ArgSlice ProcessRespSingleTokenOutput(GarnetObjectStoreOutput output) + unsafe PinnedSpanByte ProcessRespSingleTokenOutput(GarnetObjectStoreOutput output) { byte* element = null; var len = 0; - ArgSlice result; + PinnedSpanByte result; - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { fixed (byte* outputPtr = outputSpan) @@ -519,7 +514,7 @@ unsafe ArgSlice ProcessRespSingleTokenOutput(GarnetObjectStoreOutput output) if (!RespReadUtils.TryReadPtrWithLengthHeader(ref element, ref len, ref refPtr, outputPtr + outputSpan.Length)) return default; - result = new ArgSlice(element, len); + result = PinnedSpanByte.FromPinnedPointer(element, len); } } finally @@ -539,8 +534,7 @@ unsafe ArgSlice ProcessRespSingleTokenOutput(GarnetObjectStoreOutput output) /// integer unsafe bool TryProcessRespSimple64IntOutput(GarnetObjectStoreOutput output, out long value) { - var outputSpan = output.SpanByteAndMemory.IsSpanByte ? - output.SpanByteAndMemory.SpanByte.AsReadOnlySpan() : output.SpanByteAndMemory.AsMemoryReadOnlySpan(); + var outputSpan = output.SpanByteAndMemory.ReadOnlySpan; try { fixed (byte* outputPtr = outputSpan) @@ -569,18 +563,18 @@ unsafe bool TryProcessRespSimple64IntOutput(GarnetObjectStoreOutput output, out /// /// /// - unsafe GarnetStatus ReadObjectStoreOperation(byte[] key, ArgSlice input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); - ref var _input = ref Unsafe.AsRef(input.ptr); + ref var _input = ref Unsafe.AsRef(input.ToPointer()); var _output = new GarnetObjectStoreOutput(); // Perform Read on object store - var status = objectStoreContext.Read(ref key, ref _input, ref _output); + var status = objectStoreContext.Read(key, ref _input, ref _output); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref _output, ref objectStoreContext); @@ -605,8 +599,8 @@ unsafe GarnetStatus ReadObjectStoreOperation(byte[] key, ArgSlic /// /// /// - unsafe GarnetStatus ReadObjectStoreOperation(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + unsafe GarnetStatus ReadObjectStoreOperation(ReadOnlySpan key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { if (objectStoreContext.Session is null) ThrowObjectStoreUninitializedException(); @@ -614,7 +608,7 @@ unsafe GarnetStatus ReadObjectStoreOperation(byte[] key, ref Obj var _output = new GarnetObjectStoreOutput(); // Perform Read on object store - var status = objectStoreContext.Read(ref key, ref input, ref _output); + var status = objectStoreContext.Read(key, ref input, ref _output); if (status.IsPending) CompletePendingForObjectStoreSession(ref status, ref _output, ref objectStoreContext); @@ -638,8 +632,8 @@ unsafe GarnetStatus ReadObjectStoreOperation(byte[] key, ref Obj /// /// /// - public GarnetStatus ObjectScan(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus ObjectScan(ReadOnlySpan key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); [MethodImpl(MethodImplOptions.NoInlining)] @@ -657,7 +651,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); @@ -681,13 +675,11 @@ private GarnetStatus CompletePendingAndGetGarnetStatus(Status st /// The input object for the operation. /// The context of the object store. /// The status of the operation. - private GarnetStatus ObjectCollect(ArgSlice searchKey, ReadOnlySpan typeObject, SingleWriterMultiReaderLock collectLock, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + private GarnetStatus ObjectCollect(PinnedSpanByte searchKey, ReadOnlySpan typeObject, SingleWriterMultiReaderLock collectLock, ref ObjectInput input, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { if (!collectLock.TryWriteLock()) - { return GarnetStatus.NOTFOUND; - } try { @@ -697,14 +689,10 @@ private GarnetStatus ObjectCollect(ArgSlice searchKey, ReadOnlyS do { if (!DbScan(searchKey, true, cursor, out storeCursor, out var hashKeys, 100, typeObject)) - { return GarnetStatus.OK; - } foreach (var hashKey in hashKeys) - { RMWObjectStoreOperation(hashKey, ref input, out _, ref objectContext); - } cursor = storeCursor; } while (storeCursor != 0); diff --git a/libs/server/Storage/Session/ObjectStore/CompletePending.cs b/libs/server/Storage/Session/ObjectStore/CompletePending.cs index 35297661122..932f5166e6a 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = 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 a807cc4af99..4cd3b78a17b 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Server API methods - HASH @@ -32,8 +32,8 @@ sealed partial class StorageSession : IDisposable /// /// /// - public unsafe GarnetStatus HashSet(ArgSlice key, ArgSlice field, ArgSlice value, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashSet(PinnedSpanByte key, PinnedSpanByte field, PinnedSpanByte value, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -47,7 +47,7 @@ public unsafe GarnetStatus HashSet(ArgSlice key, ArgSlice field, var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HSET }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); itemsDoneCount = output.result1; return status; @@ -64,8 +64,8 @@ public unsafe GarnetStatus HashSet(ArgSlice key, ArgSlice field, /// /// /// - public unsafe GarnetStatus HashSet(ArgSlice key, (ArgSlice field, ArgSlice value)[] elements, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpanByte value)[] elements, out int itemsDoneCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -84,7 +84,7 @@ public unsafe GarnetStatus HashSet(ArgSlice key, (ArgSlice field var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HSET }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); itemsDoneCount = output.result1; return status; @@ -100,8 +100,8 @@ public unsafe GarnetStatus HashSet(ArgSlice key, (ArgSlice field /// /// /// - public GarnetStatus HashDelete(ArgSlice key, ArgSlice field, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + public GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte field, out int itemsDoneCount, ref TObjectContext objectStoreContext, bool nx = false) + where TObjectContext : ITsavoriteContext => HashDelete(key, [field], out itemsDoneCount, ref objectStoreContext); /// @@ -113,8 +113,8 @@ public GarnetStatus HashDelete(ArgSlice key, ArgSlice field, out /// /// /// - public unsafe GarnetStatus HashDelete(ArgSlice key, ArgSlice[] fields, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashDelete(PinnedSpanByte key, PinnedSpanByte[] fields, out int itemsDoneCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -128,7 +128,7 @@ public unsafe GarnetStatus HashDelete(ArgSlice key, ArgSlice[] f var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HDEL }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); itemsDoneCount = output.result1; return status; @@ -143,8 +143,8 @@ public unsafe GarnetStatus HashDelete(ArgSlice key, ArgSlice[] f /// /// /// - public unsafe GarnetStatus HashGet(ArgSlice key, ArgSlice field, out ArgSlice value, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashGet(PinnedSpanByte key, PinnedSpanByte field, out PinnedSpanByte value, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { value = default; @@ -160,7 +160,7 @@ public unsafe GarnetStatus HashGet(ArgSlice key, ArgSlice field, var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); value = default; if (status == GarnetStatus.OK) @@ -178,8 +178,8 @@ public unsafe GarnetStatus HashGet(ArgSlice key, ArgSlice field, /// /// /// - public unsafe GarnetStatus HashGetMultiple(ArgSlice key, ArgSlice[] fields, out ArgSlice[] values, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashGetMultiple(PinnedSpanByte key, PinnedSpanByte[] fields, out PinnedSpanByte[] values, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { values = default; @@ -195,7 +195,7 @@ public unsafe GarnetStatus HashGetMultiple(ArgSlice key, ArgSlic var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); values = default; if (status == GarnetStatus.OK) @@ -212,8 +212,8 @@ public unsafe GarnetStatus HashGetMultiple(ArgSlice key, ArgSlic /// /// /// - public unsafe GarnetStatus HashGetAll(ArgSlice key, out ArgSlice[] values, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashGetAll(PinnedSpanByte key, out PinnedSpanByte[] values, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { values = default; @@ -226,7 +226,7 @@ public unsafe GarnetStatus HashGetAll(ArgSlice key, out ArgSlice var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); values = default; if (status == GarnetStatus.OK) @@ -244,8 +244,8 @@ public unsafe GarnetStatus HashGetAll(ArgSlice key, out ArgSlice /// /// /// - public unsafe GarnetStatus HashLength(ArgSlice key, out int items, ref TObjectContext objectStoreContext, bool nx = false) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashLength(PinnedSpanByte key, out int items, ref TObjectContext objectStoreContext, bool nx = false) + where TObjectContext : ITsavoriteContext { items = 0; @@ -256,7 +256,7 @@ public unsafe GarnetStatus HashLength(ArgSlice key, out int item var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HLEN }; var input = new ObjectInput(header); - var status = ReadObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); items = output.result1; @@ -272,8 +272,8 @@ public unsafe GarnetStatus HashLength(ArgSlice key, out int item /// /// /// - public unsafe GarnetStatus HashExists(ArgSlice key, ArgSlice field, out bool exists, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out bool exists, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { exists = false; if (key.Length == 0) @@ -286,7 +286,7 @@ public unsafe GarnetStatus HashExists(ArgSlice key, ArgSlice fie var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HEXISTS }; var input = new ObjectInput(header, ref parseState); - var status = ReadObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); exists = output.result1 == 1; @@ -301,8 +301,8 @@ public unsafe GarnetStatus HashExists(ArgSlice key, ArgSlice fie /// /// /// - public unsafe GarnetStatus HashRandomField(ArgSlice key, out ArgSlice field, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, out PinnedSpanByte field, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { field = default; @@ -318,7 +318,7 @@ public unsafe GarnetStatus HashRandomField(ArgSlice key, out Arg var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); // Process output if (status == GarnetStatus.OK) @@ -339,8 +339,8 @@ public unsafe GarnetStatus HashRandomField(ArgSlice key, out Arg /// /// /// - public unsafe GarnetStatus HashRandomField(ArgSlice key, int count, bool withValues, out ArgSlice[] fields, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashRandomField(PinnedSpanByte key, int count, bool withValues, out PinnedSpanByte[] fields, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { fields = default; @@ -356,7 +356,7 @@ public unsafe GarnetStatus HashRandomField(ArgSlice key, int cou var input = new ObjectInput(header, inputArg, seed); var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); fields = default; if (status == GarnetStatus.OK) @@ -376,9 +376,9 @@ public unsafe GarnetStatus HashRandomField(ArgSlice key, int cou /// /// /// - public GarnetStatus HashSet(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// HashGet: Returns the value associated with field in the hash stored at key. @@ -392,9 +392,9 @@ public GarnetStatus HashSet(byte[] key, ref ObjectInput input, o /// /// /// - public GarnetStatus HashGet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus HashGet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns all fields and values of the hash stored at key. @@ -405,9 +405,9 @@ public GarnetStatus HashGet(byte[] key, ref ObjectInput input, r /// /// /// - public GarnetStatus HashGetAll(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus HashGetAll(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns the values associated with the specified fields in the hash stored at key. @@ -418,9 +418,9 @@ public GarnetStatus HashGetAll(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus HashGetMultiple(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns a random field from the hash value stored at key. @@ -431,9 +431,9 @@ public GarnetStatus HashGetMultiple(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus HashRandomField(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns the number of fields contained in the hash key. @@ -444,9 +444,9 @@ public GarnetStatus HashRandomField(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus HashLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Returns the string length of the value associated with field in the hash stored at key. If the key or the field do not exist, 0 is returned. @@ -457,9 +457,9 @@ public GarnetStatus HashLength(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus HashStrLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Removes the specified fields from the hash key. @@ -470,9 +470,9 @@ public GarnetStatus HashStrLength(byte[] key, ref ObjectInput in /// /// /// - public GarnetStatus HashDelete(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Returns if field exists in the hash stored at key. @@ -483,9 +483,9 @@ public GarnetStatus HashDelete(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus HashExists(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Returns all field names in the hash key. @@ -496,9 +496,9 @@ public GarnetStatus HashExists(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus HashKeys(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus HashKeys(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns all values in the hash key. @@ -509,9 +509,9 @@ public GarnetStatus HashKeys(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus HashVals(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Increments the number stored at field in the hash stored at key by increment. @@ -522,9 +522,9 @@ public GarnetStatus HashVals(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus HashIncrement(byte[] key, ArgSlice input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, input, out output, ref objectContext); + public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, input, out output, ref objectContext); /// /// HashIncrementByFloat: Increment the specified field of a hash stored at key, @@ -536,9 +536,9 @@ public GarnetStatus HashIncrement(byte[] key, ArgSlice input, ou /// /// /// - public GarnetStatus HashIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus HashIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Sets the expiration time for the specified key. @@ -549,11 +549,11 @@ public GarnetStatus HashIncrement(byte[] key, ref ObjectInput in /// Indicates whether the expiration time is in milliseconds. /// The expiration option to use. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus HashExpire(ArgSlice key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus HashExpire(PinnedSpanByte key, long expireAt, bool isMilliseconds, ExpireOption expireOption, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { var expireAtUtc = isMilliseconds ? ConvertUtils.UnixTimestampInMillisecondsToTicks(expireAt) : ConvertUtils.UnixTimestampInSecondsToTicks(expireAt); var expiryLength = NumUtils.CountDigits(expireAtUtc); @@ -567,7 +567,7 @@ public GarnetStatus HashExpire(ArgSlice key, long expireAt, bool var innerInput = new ObjectInput(input.header, ref parseState, startIdx: 0, arg1: (int)expireOption); - return RMWObjectStoreOperationWithOutput(key.ToArray(), ref innerInput, ref objectContext, ref output); + return RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref innerInput, ref objectContext, ref output); } /// @@ -578,15 +578,15 @@ public GarnetStatus HashExpire(ArgSlice key, long expireAt, bool /// Indicates whether the TTL is in milliseconds. /// Indicates whether the TTL is a timestamp. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus HashTimeToLive(ArgSlice key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus HashTimeToLive(PinnedSpanByte key, bool isMilliseconds, bool isTimestamp, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { var innerInput = new ObjectInput(input.header, ref input.parseState, arg1: isMilliseconds ? 1 : 0, arg2: isTimestamp ? 1 : 0); - return ReadObjectStoreOperationWithOutput(key.ToArray(), ref innerInput, ref objectContext, ref output); + return ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref innerInput, ref objectContext, ref output); } /// @@ -595,12 +595,12 @@ public GarnetStatus HashTimeToLive(ArgSlice key, bool isMillisec /// The type of the object context. /// The key of the hash. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus HashPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); + public GarnetStatus HashPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Collects hash keys and performs a specified operation on them. @@ -614,18 +614,14 @@ public GarnetStatus HashPersist(ArgSlice key, ref ObjectInput in /// If the first key is "*", all hash keys are scanned in batches and the operation is performed on each key. /// Otherwise, the operation is performed on the specified keys. /// - public GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus HashCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { if (keys[0].ReadOnlySpan.SequenceEqual("*"u8)) - { return ObjectCollect(keys[0], CmdStrings.HASH, _hcollectTaskLock, ref input, ref objectContext); - } foreach (var key in keys) - { RMWObjectStoreOperation(key.ToArray(), ref input, out _, ref objectContext); - } return GarnetStatus.OK; } diff --git a/libs/server/Storage/Session/ObjectStore/ListOps.cs b/libs/server/Storage/Session/ObjectStore/ListOps.cs index ce670688f9c..fc2d2864dd7 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -25,8 +25,8 @@ sealed partial class StorageSession : IDisposable /// The length of the list after the push operations. /// /// - public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice[] elements, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSpanByte[] elements, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -40,11 +40,10 @@ public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice[] ele var header = new RespInputHeader(GarnetObjectType.List) { ListOp = lop }; var input = new ObjectInput(header, ref parseState); - var arrKey = key.ToArray(); - var status = RMWObjectStoreOperation(arrKey, ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); itemsDoneCount = output.result1; - itemBroker.HandleCollectionUpdate(arrKey); + itemBroker.HandleCollectionUpdate(key.ToArray()); return status; } @@ -61,8 +60,8 @@ public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice[] ele /// /// /// - public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice element, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ListPush(PinnedSpanByte key, PinnedSpanByte element, ListOperation lop, out int itemsDoneCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { itemsDoneCount = 0; @@ -73,7 +72,7 @@ public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice eleme var header = new RespInputHeader(GarnetObjectType.List) { ListOp = lop }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); itemsDoneCount = output.result1; itemBroker.HandleCollectionUpdate(key.Span.ToArray()); @@ -90,8 +89,8 @@ public unsafe GarnetStatus ListPush(ArgSlice key, ArgSlice eleme /// /// /// The popped element - public GarnetStatus ListPop(ArgSlice key, ListOperation lop, ref TObjectContext objectStoreContext, out ArgSlice element) - where TObjectContext : ITsavoriteContext + public GarnetStatus ListPop(PinnedSpanByte key, ListOperation lop, ref TObjectContext objectStoreContext, out PinnedSpanByte element) + where TObjectContext : ITsavoriteContext { var status = ListPop(key, 1, lop, ref objectStoreContext, out var elements); element = elements.FirstOrDefault(); @@ -109,8 +108,8 @@ public GarnetStatus ListPop(ArgSlice key, ListOperation lop, ref /// /// /// The count elements popped from the list - public unsafe GarnetStatus ListPop(ArgSlice key, int count, ListOperation lop, ref TObjectContext objectStoreContext, out ArgSlice[] elements) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ListPop(PinnedSpanByte key, int count, ListOperation lop, ref TObjectContext objectStoreContext, out PinnedSpanByte[] elements) + where TObjectContext : ITsavoriteContext { // Prepare the input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = lop }; @@ -118,7 +117,7 @@ public unsafe GarnetStatus ListPop(ArgSlice key, int count, List var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); //process output elements = default; @@ -140,8 +139,8 @@ public unsafe GarnetStatus ListPop(ArgSlice key, int count, List /// /// /// The count elements popped from the list - public unsafe GarnetStatus ListPopMultiple(ArgSlice[] keys, OperationDirection direction, int count, ref TObjectContext objectContext, out ArgSlice key, out ArgSlice[] elements) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ListPopMultiple(PinnedSpanByte[] keys, OperationDirection direction, int count, ref TObjectContext objectContext, out PinnedSpanByte key, out PinnedSpanByte[] elements) + where TObjectContext : ITsavoriteContext { foreach (var k in keys) { @@ -175,8 +174,8 @@ public unsafe GarnetStatus ListPopMultiple(ArgSlice[] keys, Oper /// /// /// - public unsafe GarnetStatus ListLength(ArgSlice key, ref TObjectContext objectStoreContext, out int count) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref TObjectContext objectStoreContext, out int count) + where TObjectContext : ITsavoriteContext { count = 0; @@ -187,7 +186,7 @@ public unsafe GarnetStatus ListLength(ArgSlice key, ref TObjectC var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LLEN }; var input = new ObjectInput(header); - var status = ReadObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); count = output.result1; return status; @@ -203,33 +202,33 @@ public unsafe GarnetStatus ListLength(ArgSlice key, ref TObjectC /// /// out parameter, The element being popped and pushed /// GarnetStatus - public GarnetStatus ListMove(ArgSlice sourceKey, ArgSlice destinationKey, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element) + public GarnetStatus ListMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, OperationDirection sourceDirection, OperationDirection destinationDirection, out byte[] element) { element = default; - var objectLockableContext = txnManager.ObjectStoreLockableContext; + var objectTransactionalContext = txnManager.ObjectStoreTransactionalContext; - if (objectLockableContext.Session is null) + if (objectTransactionalContext.Session is null) ThrowObjectStoreUninitializedException(); // If source and destination are the same, the operation is equivalent to removing the last element from the list // and pushing it as first element of the list, so it can be considered as a list rotation command. - bool sameKey = sourceKey.ReadOnlySpan.SequenceEqual(destinationKey.ReadOnlySpan); + var sameKey = sourceKey.ReadOnlySpan.SequenceEqual(destinationKey.ReadOnlySpan); - bool createTransaction = false; + var createTransaction = false; if (txnManager.state != TxnState.Running) { createTransaction = true; txnManager.SaveKeyEntryToLock(sourceKey, true, LockType.Exclusive); txnManager.SaveKeyEntryToLock(destinationKey, true, LockType.Exclusive); - txnManager.Run(true); + _ = txnManager.Run(true); } - var objectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { // Get the source key - var statusOp = GET(sourceKey.ToArray(), out var sourceList, ref objectLockableContext); + var statusOp = GET(sourceKey, out var sourceList, ref objectTransactionalContext); if (statusOp == GarnetStatus.NOTFOUND) { @@ -247,8 +246,7 @@ public GarnetStatus ListMove(ArgSlice sourceKey, ArgSlice destinationKey, Operat if (!sameKey) { // Read destination key - var arrDestKey = destinationKey.ToArray(); - statusOp = GET(arrDestKey, out var destinationList, ref objectStoreLockableContext); + statusOp = GET(destinationKey, out var destinationList, ref objectStoreTransactionalContext); if (statusOp == GarnetStatus.NOTFOUND) { @@ -290,28 +288,28 @@ public GarnetStatus ListMove(ArgSlice sourceKey, ArgSlice destinationKey, Operat if (srcListObject.LnkList.Count == 0) { _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref objectLockableContext); + ref transactionalContext, ref objectTransactionalContext); } // Left push (addfirst) to destination if (destinationDirection == OperationDirection.Left) - dstListObject.LnkList.AddFirst(element); + _ = dstListObject.LnkList.AddFirst(element); else - dstListObject.LnkList.AddLast(element); + _ = dstListObject.LnkList.AddLast(element); dstListObject.UpdateSize(element); - newListValue = new ListObject(dstListObject.LnkList, dstListObject.Expiration, dstListObject.Size); + newListValue = new ListObject(dstListObject.LnkList, dstListObject.sizes); // Upsert - SET(destinationKey.ToArray(), newListValue, ref objectStoreLockableContext); + _ = SET(destinationKey, newListValue, ref objectStoreTransactionalContext); } else { // When the source and the destination key is the same the operation is done only in the sourceList if (destinationDirection == OperationDirection.Left) - srcListObject.LnkList.AddFirst(element); + _ = srcListObject.LnkList.AddFirst(element); else if (destinationDirection == OperationDirection.Right) - srcListObject.LnkList.AddLast(element); + _ = srcListObject.LnkList.AddLast(element); newListValue = srcListObject; ((ListObject)newListValue).UpdateSize(element); } @@ -336,14 +334,14 @@ public GarnetStatus ListMove(ArgSlice sourceKey, ArgSlice destinationKey, Operat /// /// /// true when successful - public unsafe bool ListTrim(ArgSlice key, int start, int stop, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe bool ListTrim(PinnedSpanByte key, int start, int stop, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { // Prepare the input var header = new RespInputHeader(GarnetObjectType.List) { ListOp = ListOperation.LTRIM }; var input = new ObjectInput(header, start, stop); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out _, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out _, ref objectStoreContext); return status == GarnetStatus.OK; } @@ -357,11 +355,11 @@ public unsafe bool ListTrim(ArgSlice key, int start, int stop, r /// /// /// - public GarnetStatus ListPush(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus ListPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - var status = RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); - itemBroker.HandleCollectionUpdate(key); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); + itemBroker.HandleCollectionUpdate(key.ToArray()); return status; } @@ -375,10 +373,10 @@ public GarnetStatus ListPush(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus ListPosition(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - return ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + return ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); } /// @@ -389,9 +387,9 @@ public GarnetStatus ListPosition(byte[] key, ref ObjectInput inp /// /// /// - public GarnetStatus ListTrim(byte[] key, ref ObjectInput input, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out _, ref objectStoreContext); + public GarnetStatus ListTrim(PinnedSpanByte key, ref ObjectInput input, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out _, ref objectStoreContext); /// /// Gets the specified elements of the list stored at key. @@ -402,9 +400,9 @@ public GarnetStatus ListTrim(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus ListRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Inserts a new element in the list stored at key either before or after a value pivot @@ -415,11 +413,11 @@ public GarnetStatus ListRange(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus ListInsert(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - var status = RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); - itemBroker.HandleCollectionUpdate(key); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); + itemBroker.HandleCollectionUpdate(key.ToArray()); return status; } @@ -432,9 +430,9 @@ public GarnetStatus ListInsert(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus ListIndex(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Removes the first count occurrences of elements equal to element from the list. @@ -446,9 +444,9 @@ public GarnetStatus ListIndex(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus ListRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Removes the count elements from the head(left) or tail(right) of the list stored at key. @@ -460,9 +458,9 @@ public GarnetStatus ListRemove(byte[] key, ref ObjectInput input /// /// /// - public unsafe GarnetStatus ListPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public unsafe GarnetStatus ListPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Removes the count elements from the head(left) or tail(right) of the list stored at key. @@ -474,9 +472,9 @@ public unsafe GarnetStatus ListPop(byte[] key, ref ObjectInput i /// /// /// - public unsafe GarnetStatus ListLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public unsafe GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Sets the list element at index to element. @@ -487,8 +485,8 @@ public unsafe GarnetStatus ListLength(byte[] key, ref ObjectInpu /// /// /// - public unsafe GarnetStatus ListSet(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public unsafe GarnetStatus ListSet(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + 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 81df138ca42..394f2055ab1 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Server session for RESP protocol - SET @@ -27,8 +27,8 @@ sealed partial class StorageSession : IDisposable /// /// /// - internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice member, out int saddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte member, out int saddCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { saddCount = 0; @@ -39,7 +39,7 @@ internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice membe var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SADD }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); saddCount = output.result1; return status; @@ -56,8 +56,8 @@ internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice membe /// /// /// - internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice[] members, out int saddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int saddCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { saddCount = 0; @@ -74,7 +74,7 @@ internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice[] mem // Iterate through all inputs and add them to the scratch buffer in RESP format - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); saddCount = output.result1; return status; @@ -90,8 +90,8 @@ internal unsafe GarnetStatus SetAdd(ArgSlice key, ArgSlice[] mem /// /// /// - internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice member, out int sremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte member, out int sremCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { sremCount = 0; @@ -102,7 +102,7 @@ internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice me var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SREM }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); sremCount = output.result1; return status; @@ -120,8 +120,8 @@ internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice me /// /// /// - internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice[] members, out int sremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int sremCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { sremCount = 0; @@ -135,7 +135,7 @@ internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice[] var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SREM }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); sremCount = output.result1; return status; @@ -149,8 +149,8 @@ internal unsafe GarnetStatus SetRemove(ArgSlice key, ArgSlice[] /// /// /// - internal unsafe GarnetStatus SetLength(ArgSlice key, out int count, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetLength(PinnedSpanByte key, out int count, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { count = 0; @@ -161,7 +161,7 @@ internal unsafe GarnetStatus SetLength(ArgSlice key, out int cou var header = new RespInputHeader(GarnetObjectType.Set) { SetOp = SetOperation.SCARD }; var input = new ObjectInput(header); - var status = ReadObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); count = output.result1; return status; @@ -175,8 +175,8 @@ internal unsafe GarnetStatus SetLength(ArgSlice key, out int cou /// /// /// - internal unsafe GarnetStatus SetMembers(ArgSlice key, out ArgSlice[] members, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetMembers(PinnedSpanByte key, out PinnedSpanByte[] members, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { members = default; @@ -189,7 +189,7 @@ internal unsafe GarnetStatus SetMembers(ArgSlice key, out ArgSli var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); if (status == GarnetStatus.OK) members = ProcessRespArrayOutput(output, out _); @@ -205,8 +205,8 @@ internal unsafe GarnetStatus SetMembers(ArgSlice key, out ArgSli /// /// /// - internal GarnetStatus SetPop(ArgSlice key, out ArgSlice element, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal GarnetStatus SetPop(PinnedSpanByte key, out PinnedSpanByte element, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { var status = SetPop(key, int.MinValue, out var elements, ref objectStoreContext); element = default; @@ -225,8 +225,8 @@ internal GarnetStatus SetPop(ArgSlice key, out ArgSlice element, /// /// /// - internal unsafe GarnetStatus SetPop(ArgSlice key, int count, out ArgSlice[] elements, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + internal unsafe GarnetStatus SetPop(PinnedSpanByte key, int count, out PinnedSpanByte[] elements, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { elements = default; @@ -239,7 +239,7 @@ internal unsafe GarnetStatus SetPop(ArgSlice key, int count, out var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); if (status != GarnetStatus.OK) return status; @@ -259,7 +259,7 @@ internal unsafe GarnetStatus SetPop(ArgSlice key, int count, out /// /// /// - internal unsafe GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey, ArgSlice member, out int smoveResult) + internal unsafe GarnetStatus SetMove(PinnedSpanByte sourceKey, PinnedSpanByte destinationKey, PinnedSpanByte member, out int smoveResult) { smoveResult = 0; @@ -272,14 +272,11 @@ internal unsafe GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey _ = txnManager.Run(true); } - var objectLockableContext = txnManager.ObjectStoreLockableContext; + var objectTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - var arrDstKey = destinationKey.ToArray(); - var arrSrcKey = sourceKey.ToArray(); - - var srcGetStatus = GET(arrSrcKey, out var srcObject, ref objectLockableContext); + var srcGetStatus = GET(sourceKey, out var srcObject, ref objectTransactionalContext); if (srcGetStatus == GarnetStatus.NOTFOUND) return GarnetStatus.NOTFOUND; @@ -292,7 +289,7 @@ internal unsafe GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey if (sameKey) return GarnetStatus.OK; - var dstGetStatus = GET(arrDstKey, out var dstObject, ref objectLockableContext); + var dstGetStatus = GET(destinationKey, out var dstObject, ref objectTransactionalContext); SetObject dstSetObject; if (dstGetStatus == GarnetStatus.OK) @@ -317,15 +314,15 @@ internal unsafe GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey if (srcSetObject.Set.Count == 0) { _ = EXPIRE(sourceKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref objectLockableContext); + ref transactionalContext, ref objectTransactionalContext); } - dstSetObject.Set.Add(arrMember); + _ = dstSetObject.Set.Add(arrMember); dstSetObject.UpdateSize(arrMember); if (dstGetStatus == GarnetStatus.NOTFOUND) { - var setStatus = SET(arrDstKey, dstSetObject, ref objectLockableContext); + var setStatus = SET(destinationKey, dstSetObject, ref objectTransactionalContext); if (setStatus == GarnetStatus.OK) smoveResult = 1; } @@ -351,7 +348,7 @@ internal unsafe GarnetStatus SetMove(ArgSlice sourceKey, ArgSlice destinationKey /// /// /// - public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetIntersect(PinnedSpanByte[] keys, out HashSet output) { output = default; @@ -370,11 +367,11 @@ public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - return SetIntersect(keys, ref setObjectStoreLockableContext, out output); + return SetIntersect(keys, ref setObjectStoreTransactionalContext, out output); } finally { @@ -391,7 +388,7 @@ public GarnetStatus SetIntersect(ArgSlice[] keys, out HashSet output) /// /// /// - public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetIntersectStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) { count = default; @@ -400,26 +397,24 @@ public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count return GarnetStatus.OK; } - var destination = scratchBufferManager.CreateArgSlice(key); - var createTransaction = false; if (txnManager.state != TxnState.Running) { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destination, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); foreach (var item in keys) txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); _ = txnManager.Run(true); } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - var status = SetIntersect(keys, ref setObjectStoreLockableContext, out var members); + var status = SetIntersect(keys, ref setObjectStoreTransactionalContext, out var members); if (status == GarnetStatus.OK) { @@ -432,12 +427,12 @@ public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count newSetObject.UpdateSize(item); } - _ = SET(key, newSetObject, ref setObjectStoreLockableContext); + _ = SET(key, newSetObject, ref setObjectStoreTransactionalContext); } else { - _ = EXPIRE(destination, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref setObjectStoreLockableContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, + ref transactionalContext, ref setObjectStoreTransactionalContext); } count = members.Count; @@ -453,8 +448,8 @@ public GarnetStatus SetIntersectStore(byte[] key, ArgSlice[] keys, out int count } - private GarnetStatus SetIntersect(ReadOnlySpan keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + private GarnetStatus SetIntersect(ReadOnlySpan keys, ref TObjectContext objectContext, out HashSet output) + where TObjectContext : ITsavoriteContext { output = new HashSet(ByteArrayComparer.Instance); @@ -463,17 +458,11 @@ private GarnetStatus SetIntersect(ReadOnlySpan keys, r return GarnetStatus.OK; } - var status = GET(keys[0].ToArray(), out var first, ref objectContext); - + var status = GET(keys[0], out var first, ref objectContext); if (status == GarnetStatus.NOTFOUND) - { return GarnetStatus.OK; - } - if (status == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } if (status == GarnetStatus.OK) { @@ -500,7 +489,7 @@ private GarnetStatus SetIntersect(ReadOnlySpan keys, r return GarnetStatus.OK; } - status = GET(keys[i].ToArray(), out var next, ref objectContext); + status = GET(keys[i], out var next, ref objectContext); if (status == GarnetStatus.WRONGTYPE) return GarnetStatus.WRONGTYPE; if (status == GarnetStatus.OK) @@ -530,7 +519,7 @@ private GarnetStatus SetIntersect(ReadOnlySpan keys, r /// /// /// - public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) + public GarnetStatus SetUnion(PinnedSpanByte[] keys, out HashSet output) { output = new HashSet(ByteArrayComparer.Instance); @@ -549,11 +538,11 @@ public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - return SetUnion(keys, ref setObjectStoreLockableContext, out output); + return SetUnion(keys, ref setObjectStoreTransactionalContext, out output); } finally { @@ -570,33 +559,31 @@ public GarnetStatus SetUnion(ArgSlice[] keys, out HashSet output) /// /// /// - public GarnetStatus SetUnionStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetUnionStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) { count = default; if (keys.Length == 0) return GarnetStatus.OK; - var destination = scratchBufferManager.CreateArgSlice(key); - var createTransaction = false; if (txnManager.state != TxnState.Running) { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destination, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); foreach (var item in keys) txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); _ = txnManager.Run(true); } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - var status = SetUnion(keys, ref setObjectStoreLockableContext, out var members); + var status = SetUnion(keys, ref setObjectStoreTransactionalContext, out var members); if (status == GarnetStatus.OK) { @@ -609,12 +596,12 @@ public GarnetStatus SetUnionStore(byte[] key, ArgSlice[] keys, out int count) newSetObject.UpdateSize(item); } - _ = SET(key, newSetObject, ref setObjectStoreLockableContext); + _ = SET(key, newSetObject, ref setObjectStoreTransactionalContext); } else { - _ = EXPIRE(destination, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref setObjectStoreLockableContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, + ref transactionalContext, ref setObjectStoreTransactionalContext); } count = members.Count; @@ -629,18 +616,16 @@ public GarnetStatus SetUnionStore(byte[] key, ArgSlice[] keys, out int count) } } - private GarnetStatus SetUnion(ArgSlice[] keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + private GarnetStatus SetUnion(PinnedSpanByte[] keys, ref TObjectContext objectContext, out HashSet output) + where TObjectContext : ITsavoriteContext { output = new HashSet(ByteArrayComparer.Instance); if (keys.Length == 0) - { return GarnetStatus.OK; - } foreach (var item in keys) { - if (GET(item.ToArray(), out var currObject, ref objectContext) == GarnetStatus.OK) + if (GET(item, out var currObject, ref objectContext) == GarnetStatus.OK) { if (currObject.GarnetObject is not SetObject setObject) { @@ -666,9 +651,9 @@ private GarnetStatus SetUnion(ArgSlice[] keys, ref TObjectContex /// /// /// - public GarnetStatus SetAdd(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectContext); + public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// /// Removes the specified members from the set. @@ -681,9 +666,9 @@ public GarnetStatus SetAdd(byte[] key, ref ObjectInput input, ou /// /// /// - public GarnetStatus SetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectContext); + public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// /// Returns the number of elements of the set. @@ -694,9 +679,9 @@ public GarnetStatus SetRemove(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus SetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectContext); + public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// /// Returns all members of the set at key. @@ -707,9 +692,9 @@ public GarnetStatus SetLength(byte[] key, ref ObjectInput input, /// /// /// - public GarnetStatus SetMembers(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SetMembers(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns if member is a member of the set stored at key. @@ -720,9 +705,9 @@ public GarnetStatus SetMembers(byte[] key, ref ObjectInput input /// /// /// - public GarnetStatus SetIsMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SetIsMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns whether each member is a member of the set stored at key. @@ -732,8 +717,8 @@ public GarnetStatus SetIsMember(byte[] key, ref ObjectInput inpu /// /// /// - public unsafe GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] members, out int[] result, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SetIsMember(PinnedSpanByte key, PinnedSpanByte[] members, out int[] result, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { result = default; @@ -749,8 +734,8 @@ public unsafe GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] SetOp = SetOperation.SMISMEMBER, }, ref parseState); - var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); + var output = new GarnetObjectStoreOutput { SpanByteAndMemory = new SpanByteAndMemory(null) }; + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); if (status == GarnetStatus.OK) result = ProcessRespIntegerArrayOutput(output, out _); @@ -767,9 +752,9 @@ public unsafe GarnetStatus SetIsMember(ArgSlice key, ArgSlice[] /// /// /// - public GarnetStatus SetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// When called with just the key argument, return a random element from the set value stored at key. @@ -784,9 +769,9 @@ public GarnetStatus SetPop(byte[] key, ref ObjectInput input, re /// /// /// - public GarnetStatus SetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns the members of the set resulting from the difference between the first set at key and all the successive sets at keys. @@ -794,7 +779,7 @@ public GarnetStatus SetRandomMember(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet members) + public GarnetStatus SetDiff(PinnedSpanByte[] keys, out HashSet members) { members = default; @@ -813,11 +798,11 @@ public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet members) } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - return SetDiff(keys, ref setObjectStoreLockableContext, out members); + return SetDiff(keys, ref setObjectStoreTransactionalContext, out members); } finally { @@ -834,33 +819,31 @@ public GarnetStatus SetDiff(ArgSlice[] keys, out HashSet members) /// /// /// - public GarnetStatus SetDiffStore(byte[] key, ArgSlice[] keys, out int count) + public GarnetStatus SetDiffStore(PinnedSpanByte key, PinnedSpanByte[] keys, out int count) { count = default; if (keys.Length == 0) return GarnetStatus.OK; - var destination = scratchBufferManager.CreateArgSlice(key); - var createTransaction = false; if (txnManager.state != TxnState.Running) { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(destination, true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(key, true, LockType.Exclusive); foreach (var item in keys) txnManager.SaveKeyEntryToLock(item, true, LockType.Shared); _ = txnManager.Run(true); } // SetObject - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - var status = SetDiff(keys, ref setObjectStoreLockableContext, out var diffSet); + var status = SetDiff(keys, ref setObjectStoreTransactionalContext, out var diffSet); if (status == GarnetStatus.OK) { @@ -872,12 +855,12 @@ public GarnetStatus SetDiffStore(byte[] key, ArgSlice[] keys, out int count) _ = newSetObject.Set.Add(item); newSetObject.UpdateSize(item); } - _ = SET(key, newSetObject, ref setObjectStoreLockableContext); + _ = SET(key, newSetObject, ref setObjectStoreTransactionalContext); } else { - _ = EXPIRE(destination, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref setObjectStoreLockableContext); + _ = EXPIRE(key, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, + ref transactionalContext, ref setObjectStoreTransactionalContext); } count = diffSet.Count; @@ -892,8 +875,8 @@ public GarnetStatus SetDiffStore(byte[] key, ArgSlice[] keys, out int count) } } - private GarnetStatus SetDiff(ArgSlice[] keys, ref TObjectContext objectContext, out HashSet output) - where TObjectContext : ITsavoriteContext + private GarnetStatus SetDiff(PinnedSpanByte[] keys, ref TObjectContext objectContext, out HashSet output) + where TObjectContext : ITsavoriteContext { output = new HashSet(); if (keys.Length == 0) @@ -902,17 +885,11 @@ private GarnetStatus SetDiff(ArgSlice[] keys, ref TObjectContext } // first SetObject - var status = GET(keys[0].ToArray(), out var first, ref objectContext); - + var status = GET(keys[0], out var first, ref objectContext); if (status == GarnetStatus.NOTFOUND) - { return GarnetStatus.OK; - } - if (status == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } if (status == GarnetStatus.OK) { @@ -932,7 +909,7 @@ private GarnetStatus SetDiff(ArgSlice[] keys, ref TObjectContext // after SetObjects for (var i = 1; i < keys.Length; i++) { - status = GET(keys[i].ToArray(), out var next, ref objectContext); + status = GET(keys[i], out var next, ref objectContext); if (status == GarnetStatus.WRONGTYPE) return GarnetStatus.WRONGTYPE; if (status == GarnetStatus.OK) @@ -957,9 +934,9 @@ private GarnetStatus SetDiff(ArgSlice[] keys, ref TObjectContext /// Optional limit for stopping early when reaching this size /// /// - public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, out int count) { - if (txnManager.ObjectStoreLockableContext.Session is null) + if (txnManager.ObjectStoreTransactionalContext.Session is null) ThrowObjectStoreUninitializedException(); count = 0; @@ -978,11 +955,11 @@ public GarnetStatus SetIntersectLength(ReadOnlySpan keys, int? limit, _ = txnManager.Run(true); } - var setObjectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var setObjectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { - var status = SetIntersect(keys, ref setObjectStoreLockableContext, out var result); + var status = SetIntersect(keys, ref setObjectStoreTransactionalContext, out var result); if (status == GarnetStatus.OK && result != null) { count = limit.HasValue ? Math.Min(result.Count, limit.Value) : result.Count; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetGeoOps.cs index 5abc6c4174d..c7241c3ca38 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -23,9 +23,9 @@ sealed partial class StorageSession : IDisposable /// /// /// - public GarnetStatus GeoAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus GeoAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// GEOHASH: Returns valid Geohash strings representing the position of one or more elements in a geospatial data of the sorted set. @@ -38,9 +38,9 @@ public GarnetStatus GeoAdd(byte[] key, ref ObjectInput input, re /// /// /// - public GarnetStatus GeoCommands(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus GeoCommands(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Geospatial search and return result.. @@ -57,11 +57,11 @@ public GarnetStatus GeoCommands(byte[] key, ref ObjectInput inpu /// /// /// - public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearchOptions opts, + public GarnetStatus GeoSearchReadOnly(PinnedSpanByte key, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var createTransaction = false; @@ -76,7 +76,7 @@ public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearc try { // Can we optimize more when ANY is used? - var statusOp = GET(key.ToArray(), out var firstObj, ref objectContext); + var statusOp = GET(key, out var firstObj, ref objectContext); if (statusOp == GarnetStatus.OK) { if (firstObj.GarnetObject is not SortedSetObject firstSortedSet) @@ -113,12 +113,12 @@ public GarnetStatus GeoSearchReadOnly(ArgSlice key, ref GeoSearc /// /// /// - public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destination, + public unsafe GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destination, ref GeoSearchOptions opts, ref ObjectInput input, ref SpanByteAndMemory output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { var createTransaction = false; @@ -130,7 +130,7 @@ public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice txnManager.SaveKeyEntryToLock(key, true, LockType.Shared); _ = txnManager.Run(true); } - var objectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; using var writer = new RespMemoryWriter(functionsState.respProtocolVersion, ref output); @@ -138,18 +138,13 @@ public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice { SpanByteAndMemory searchOutMem = default; - var status = GET(key.ToArray(), out var firstObj, ref objectStoreLockableContext); + var status = GET(key, out var firstObj, ref objectStoreTransactionalContext); if (status == GarnetStatus.OK) { if (firstObj.GarnetObject is SortedSetObject firstSortedSet) - { - firstSortedSet.GeoSearch(ref input, ref searchOutMem, functionsState.respProtocolVersion, - ref opts, false); - } + firstSortedSet.GeoSearch(ref input, ref searchOutMem, functionsState.respProtocolVersion, ref opts, false); else - { status = GarnetStatus.WRONGTYPE; - } } if (status == GarnetStatus.WRONGTYPE) @@ -160,7 +155,7 @@ public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice 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 lockableContext, ref objectStoreLockableContext); + _ = EXPIRE(destination, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, ref transactionalContext, ref objectStoreTransactionalContext); writer.WriteInt32(0); return GarnetStatus.OK; } @@ -180,23 +175,22 @@ public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice return GarnetStatus.OK; } - var destinationKey = destination.ToArray(); - objectStoreLockableContext.Delete(ref destinationKey); + _ = objectStoreTransactionalContext.Delete(destination.ReadOnlySpan); - RespReadUtils.TryReadUnsignedArrayLength(out var foundItems, ref currOutPtr, endOutPtr); + _ = RespReadUtils.TryReadUnsignedArrayLength(out var foundItems, ref currOutPtr, endOutPtr); // Prepare the parse state for sorted set add parseState.Initialize(foundItems * 2); for (var j = 0; j < foundItems; j++) { - RespReadUtils.TryReadUnsignedArrayLength(out var innerLength, ref currOutPtr, endOutPtr); + _ = RespReadUtils.TryReadUnsignedArrayLength(out var innerLength, ref currOutPtr, endOutPtr); Debug.Assert(innerLength == 2, "Should always has location and hash or distance"); // Read location into parse state - parseState.Read((2 * j) + 1, ref currOutPtr, endOutPtr); + _ = parseState.Read((2 * j) + 1, ref currOutPtr, endOutPtr); // Read score into parse state - parseState.Read(2 * j, ref currOutPtr, endOutPtr); + _ = parseState.Read(2 * j, ref currOutPtr, endOutPtr); } // Prepare the input @@ -207,13 +201,14 @@ public unsafe GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice }, ref parseState); var zAddOutput = new GarnetObjectStoreOutput(); - RMWObjectStoreOperationWithOutput(destinationKey, ref zAddInput, ref objectStoreLockableContext, ref zAddOutput); + RMWObjectStoreOperationWithOutput(destination, ref zAddInput, ref objectStoreTransactionalContext, ref zAddOutput); writer.WriteInt32(foundItems); } finally { searchOutHandler.Dispose(); + searchOutMem.Dispose(); } return GarnetStatus.OK; diff --git a/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs b/libs/server/Storage/Session/ObjectStore/SortedSetOps.cs index ef4b3b21c91..03129dcc3e9 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 = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; sealed partial class StorageSession : IDisposable { @@ -29,8 +29,8 @@ sealed partial class StorageSession : IDisposable /// /// /// - public unsafe GarnetStatus SortedSetAdd(ArgSlice key, ArgSlice score, ArgSlice member, out int zaddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, PinnedSpanByte score, PinnedSpanByte member, out int zaddCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { zaddCount = 0; if (key.Length == 0) @@ -45,14 +45,11 @@ public unsafe GarnetStatus SortedSetAdd(ArgSlice key, ArgSlice s var output = new GarnetObjectStoreOutput(); - var keyBytes = key.ToArray(); - var status = RMWObjectStoreOperationWithOutput(keyBytes, ref input, ref objectStoreContext, ref output); - itemBroker.HandleCollectionUpdate(keyBytes); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); + itemBroker.HandleCollectionUpdate(key.ToArray()); if (status == GarnetStatus.OK) - { zaddCount = TryProcessRespSimple64IntOutput(output, out var value) ? (int)value : default; - } return status; } @@ -67,8 +64,8 @@ public unsafe GarnetStatus SortedSetAdd(ArgSlice key, ArgSlice s /// /// /// - public unsafe GarnetStatus SortedSetAdd(ArgSlice key, (ArgSlice score, ArgSlice member)[] inputs, out int zaddCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetAdd(PinnedSpanByte key, (PinnedSpanByte score, PinnedSpanByte member)[] inputs, out int zaddCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { zaddCount = 0; @@ -88,9 +85,8 @@ public unsafe GarnetStatus SortedSetAdd(ArgSlice key, (ArgSlice var output = new GarnetObjectStoreOutput(); - var keyBytes = key.ToArray(); - var status = RMWObjectStoreOperationWithOutput(keyBytes, ref input, ref objectStoreContext, ref output); - itemBroker.HandleCollectionUpdate(keyBytes); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); + itemBroker.HandleCollectionUpdate(key.ToArray()); if (status == GarnetStatus.OK) { @@ -110,10 +106,9 @@ public unsafe GarnetStatus SortedSetAdd(ArgSlice key, (ArgSlice /// /// /// - public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice member, out int zremCount, + public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte member, out int zremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { zremCount = 0; @@ -127,7 +122,7 @@ public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZREM }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key, ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); zremCount = output.result1; return status; @@ -143,8 +138,8 @@ public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice /// /// /// - public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice[] members, out int zremCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out int zremCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { zremCount = 0; @@ -157,7 +152,7 @@ public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice[ var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZREM }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key, ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); zremCount = output.result1; return status; @@ -173,10 +168,9 @@ public unsafe GarnetStatus SortedSetRemove(byte[] key, ArgSlice[ /// /// /// - public unsafe GarnetStatus SortedSetRemoveRangeByLex(ArgSlice key, string min, string max, + public unsafe GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, string min, string max, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -190,11 +184,11 @@ public unsafe GarnetStatus SortedSetRemoveRangeByLex(ArgSlice ke // Store parameters to buffer var minSpan = paramsSpan.Slice(0, min.Length); Encoding.UTF8.GetBytes(min, minSpan); - var minSlice = ArgSlice.FromPinnedSpan(minSpan); + var minSlice = PinnedSpanByte.FromPinnedSpan(minSpan); var maxSpan = paramsSpan.Slice(min.Length, max.Length); Encoding.UTF8.GetBytes(max, maxSpan); - var maxSlice = ArgSlice.FromPinnedSpan(maxSpan); + var maxSlice = PinnedSpanByte.FromPinnedSpan(maxSpan); // Prepare the parse state parseState.InitializeWithArguments(minSlice, maxSlice); @@ -203,10 +197,10 @@ public unsafe GarnetStatus SortedSetRemoveRangeByLex(ArgSlice ke var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZREMRANGEBYLEX }; var input = new ObjectInput(header, ref parseState); - var status = RMWObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); countRemoved = output.result1; - scratchBufferManager.RewindScratchBuffer(ref paramsSlice); + scratchBufferManager.RewindScratchBuffer(paramsSlice); return status; } @@ -221,10 +215,9 @@ public unsafe GarnetStatus SortedSetRemoveRangeByLex(ArgSlice ke /// /// /// - public unsafe GarnetStatus SortedSetRemoveRangeByScore(ArgSlice key, string min, string max, + public unsafe GarnetStatus SortedSetRemoveRangeByScore(PinnedSpanByte key, string min, string max, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -238,11 +231,11 @@ public unsafe GarnetStatus SortedSetRemoveRangeByScore(ArgSlice // Store parameters to buffer var minSpan = paramsSpan.Slice(0, min.Length); Encoding.UTF8.GetBytes(min, minSpan); - var minSlice = ArgSlice.FromPinnedSpan(minSpan); + var minSlice = PinnedSpanByte.FromPinnedSpan(minSpan); var maxSpan = paramsSpan.Slice(min.Length, max.Length); Encoding.UTF8.GetBytes(max, maxSpan); - var maxSlice = ArgSlice.FromPinnedSpan(maxSpan); + var maxSlice = PinnedSpanByte.FromPinnedSpan(maxSpan); // Prepare the parse state parseState.InitializeWithArguments(minSlice, maxSlice); @@ -253,10 +246,10 @@ public unsafe GarnetStatus SortedSetRemoveRangeByScore(ArgSlice var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); - scratchBufferManager.RewindScratchBuffer(ref paramsSlice); + scratchBufferManager.RewindScratchBuffer(paramsSlice); if (status == GarnetStatus.OK) { @@ -278,10 +271,9 @@ public unsafe GarnetStatus SortedSetRemoveRangeByScore(ArgSlice /// /// /// - public unsafe GarnetStatus SortedSetRemoveRangeByRank(ArgSlice key, int start, int stop, + public unsafe GarnetStatus SortedSetRemoveRangeByRank(PinnedSpanByte key, int start, int stop, out int countRemoved, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { countRemoved = 0; @@ -300,11 +292,11 @@ public unsafe GarnetStatus SortedSetRemoveRangeByRank(ArgSlice k // Store parameters to buffer var startSpan = paramsSpan.Slice(0, startLen); NumUtils.WriteInt64(start, startSpan); - var startSlice = ArgSlice.FromPinnedSpan(startSpan); + var startSlice = PinnedSpanByte.FromPinnedSpan(startSpan); var stopSpan = paramsSpan.Slice(startLen, stopLen); NumUtils.WriteInt64(stop, stopSpan); - var stopSlice = ArgSlice.FromPinnedSpan(stopSpan); + var stopSlice = PinnedSpanByte.FromPinnedSpan(stopSpan); parseState.InitializeWithArguments(startSlice, stopSlice); @@ -314,10 +306,10 @@ public unsafe GarnetStatus SortedSetRemoveRangeByRank(ArgSlice k var output = new GarnetObjectStoreOutput(); - status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, + status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); - scratchBufferManager.RewindScratchBuffer(ref paramsSlice); + scratchBufferManager.RewindScratchBuffer(paramsSlice); if (status == GarnetStatus.OK) { @@ -337,8 +329,8 @@ public unsafe GarnetStatus SortedSetRemoveRangeByRank(ArgSlice k /// /// /// - public unsafe GarnetStatus SortedSetPop(ArgSlice key, int count, bool lowScoresFirst, out (ArgSlice member, ArgSlice score)[] pairs, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetPop(PinnedSpanByte key, int count, bool lowScoresFirst, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { pairs = default; if (key.Length == 0) @@ -351,7 +343,7 @@ public unsafe GarnetStatus SortedSetPop(ArgSlice key, int count, var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); //process output if (status == GarnetStatus.OK) @@ -372,10 +364,9 @@ public unsafe GarnetStatus SortedSetPop(ArgSlice key, int count, /// /// /// - public unsafe GarnetStatus SortedSetIncrement(ArgSlice key, double increment, ArgSlice member, + public unsafe GarnetStatus SortedSetIncrement(PinnedSpanByte key, double increment, PinnedSpanByte member, out double newScore, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { newScore = 0; @@ -393,8 +384,8 @@ public unsafe GarnetStatus SortedSetIncrement(ArgSlice key, doub var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZINCRBY }; var input = new ObjectInput(header, ref parseState); - var output = new GarnetObjectStoreOutput(); - var status = RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, + var output = new GarnetObjectStoreOutput { SpanByteAndMemory = new SpanByteAndMemory(null) }; + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); // Process output @@ -419,8 +410,8 @@ public unsafe GarnetStatus SortedSetIncrement(ArgSlice key, doub /// /// /// - public unsafe GarnetStatus SortedSetLength(ArgSlice key, out int zcardCount, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { zcardCount = 0; @@ -431,7 +422,7 @@ public unsafe GarnetStatus SortedSetLength(ArgSlice key, out int var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZCARD }; var input = new ObjectInput(header); - var status = ReadObjectStoreOperation(key.ToArray(), ref input, out var output, ref objectStoreContext); + var status = ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out var output, ref objectStoreContext); zcardCount = output.result1; return status; @@ -454,8 +445,8 @@ public unsafe GarnetStatus SortedSetLength(ArgSlice key, out int /// /// /// - public unsafe GarnetStatus SortedSetRange(ArgSlice key, ArgSlice min, ArgSlice max, SortedSetOrderOperation sortedSetOrderOperation, ref TObjectContext objectContext, out ArgSlice[] elements, out string error, bool withScores = false, bool reverse = false, (string, int) limit = default) - where TObjectContext : ITsavoriteContext + 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 { elements = default; error = default; @@ -475,7 +466,7 @@ public unsafe GarnetStatus SortedSetRange(ArgSlice key, ArgSlice _ => SortedSetRangeOpts.None }; - var arguments = new List { min, max }; + var arguments = new List { min, max }; if (reverse) { @@ -503,13 +494,13 @@ public unsafe GarnetStatus SortedSetRange(ArgSlice key, ArgSlice var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZRANGE }; var input = new ObjectInput(header, ref parseState, arg2: (int)rangeOpts); - var output = new GarnetObjectStoreOutput(); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); + var output = new GarnetObjectStoreOutput { SpanByteAndMemory = new SpanByteAndMemory(null) }; + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); for (var i = arguments.Count - 1; i > 1; i--) { var currSlice = arguments[i]; - scratchBufferManager.RewindScratchBuffer(ref currSlice); + scratchBufferManager.RewindScratchBuffer(currSlice); } if (status == GarnetStatus.OK) @@ -525,7 +516,7 @@ public unsafe GarnetStatus SortedSetRange(ArgSlice key, ArgSlice /// /// /// - public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys, out Dictionary pairs) + public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys, out Dictionary pairs) { pairs = default; @@ -543,7 +534,7 @@ public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys, out txnManager.Run(true); } - var objectContext = txnManager.ObjectStoreLockableContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; try { @@ -563,7 +554,7 @@ public unsafe GarnetStatus SortedSetDifference(ReadOnlySpan keys, out /// /// /// - public GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySpan keys, out int count) + public GarnetStatus SortedSetDifferenceStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, out int count) { count = default; @@ -582,7 +573,7 @@ public GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySp _ = txnManager.Run(true); } - var objectContext = txnManager.ObjectStoreLockableContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; try { @@ -602,14 +593,13 @@ public GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySp newSetObject.Add(element, score); } - var destinationKeyBytes = destinationKey.ToArray(); - _ = SET(destinationKeyBytes, newSetObject, ref objectContext); - itemBroker.HandleCollectionUpdate(destinationKeyBytes); + _ = SET(destinationKey, newSetObject, ref objectContext); + itemBroker.HandleCollectionUpdate(destinationKey.ToArray()); } else { _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref objectContext); + ref transactionalContext, ref objectContext); } return status; @@ -629,8 +619,8 @@ public GarnetStatus SortedSetDifferenceStore(ArgSlice destinationKey, ReadOnlySp /// The rank of the member (null if the member does not exist) /// /// - public unsafe GarnetStatus SortedSetRank(ArgSlice key, ArgSlice member, bool reverse, out long? rank, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetRank(PinnedSpanByte key, PinnedSpanByte member, bool reverse, out long? rank, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { rank = null; if (key.Length == 0) @@ -646,9 +636,9 @@ public unsafe GarnetStatus SortedSetRank(ArgSlice key, ArgSlice const int outputContainerSize = 32; // 3 for HEADER + CRLF + 20 for ascii long var outputContainer = stackalloc byte[outputContainerSize]; - var output = new GarnetObjectStoreOutput(new(outputContainer, outputContainerSize)); + var output = GarnetObjectStoreOutput.FromPinnedPointer(outputContainer, outputContainerSize); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectStoreContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); if (status == GarnetStatus.OK) { @@ -676,11 +666,11 @@ public unsafe GarnetStatus SortedSetRank(ArgSlice key, ArgSlice /// /// /// - public GarnetStatus SortedSetAdd(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetAdd(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - var status = RMWObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); - itemBroker.HandleCollectionUpdate(key); + var status = RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); + itemBroker.HandleCollectionUpdate(key.ToArray()); return status; } @@ -694,10 +684,10 @@ public GarnetStatus SortedSetAdd(byte[] key, ref ObjectInput inp /// The result of the operation, indicating the number of elements stored. /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. - public unsafe GarnetStatus SortedSetRangeStore(ArgSlice dstKey, ArgSlice srcKey, ref ObjectInput input, out int result, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetRangeStore(PinnedSpanByte dstKey, PinnedSpanByte srcKey, ref ObjectInput input, out int result, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext { - if (txnManager.ObjectStoreLockableContext.Session is null) + if (txnManager.ObjectStoreTransactionalContext.Session is null) ThrowObjectStoreUninitializedException(); result = 0; @@ -711,31 +701,28 @@ public unsafe GarnetStatus SortedSetRangeStore(ArgSlice dstKey, { Debug.Assert(txnManager.state == TxnState.None); createTransaction = true; - txnManager.SaveKeyEntryToLock(dstKey, true, LockType.Exclusive); - txnManager.SaveKeyEntryToLock(srcKey, true, LockType.Shared); + txnManager.SaveKeyEntryToLock(dstKey, isObject: true, LockType.Exclusive); + txnManager.SaveKeyEntryToLock(srcKey, isObject: true, LockType.Shared); _ = txnManager.Run(true); } // SetObject - var objectStoreLockableContext = txnManager.ObjectStoreLockableContext; + var objectStoreTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { SpanByteAndMemory rangeOutputMem = default; var rangeOutput = new GarnetObjectStoreOutput(rangeOutputMem); - - var status = SortedSetRange(srcKey.ToArray(), ref input, ref rangeOutput, ref objectStoreLockableContext); + var status = SortedSetRange(srcKey, ref input, ref rangeOutput, ref objectStoreTransactionalContext); rangeOutputMem = rangeOutput.SpanByteAndMemory; if (status == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } 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 lockableContext, ref objectStoreLockableContext); + _ = EXPIRE(dstKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, ref transactionalContext, ref objectStoreTransactionalContext); return GarnetStatus.OK; } @@ -748,8 +735,8 @@ public unsafe GarnetStatus SortedSetRangeStore(ArgSlice dstKey, ref var currOutPtr = ref rangeOutPtr; var endOutPtr = rangeOutPtr + rangeOutputMem.Length; - var destinationKey = dstKey.ToArray(); - objectStoreLockableContext.Delete(ref destinationKey); + var destinationKey = dstKey.ReadOnlySpan; + objectStoreTransactionalContext.Delete(destinationKey); RespReadUtils.TryReadUnsignedArrayLength(out var arrayLen, ref currOutPtr, endOutPtr); Debug.Assert(arrayLen % 2 == 0, "Should always contain element and its score"); @@ -774,8 +761,8 @@ public unsafe GarnetStatus SortedSetRangeStore(ArgSlice dstKey, }, ref parseState); var zAddOutput = new GarnetObjectStoreOutput(); - RMWObjectStoreOperationWithOutput(destinationKey, ref zAddInput, ref objectStoreLockableContext, ref zAddOutput); - itemBroker.HandleCollectionUpdate(destinationKey); + RMWObjectStoreOperationWithOutput(destinationKey, ref zAddInput, ref objectStoreTransactionalContext, ref zAddOutput); + itemBroker.HandleCollectionUpdate(destinationKey.ToArray()); } } finally @@ -801,9 +788,9 @@ public unsafe GarnetStatus SortedSetRangeStore(ArgSlice dstKey, /// /// /// - public GarnetStatus SortedSetRemove(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Returns the number of members of the sorted set. @@ -814,9 +801,9 @@ public GarnetStatus SortedSetRemove(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus SortedSetLength(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Returns the specified range of elements in the sorted set stored at key. @@ -829,9 +816,9 @@ public GarnetStatus SortedSetLength(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus SortedSetRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns the score of member in the sorted set at key. @@ -843,9 +830,9 @@ public GarnetStatus SortedSetRange(byte[] key, ref ObjectInput i /// /// /// - public GarnetStatus SortedSetScore(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetScore(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns the scores of members in the sorted set at key. @@ -857,9 +844,9 @@ public GarnetStatus SortedSetScore(byte[] key, ref ObjectInput i /// /// /// - public GarnetStatus SortedSetScores(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetScores(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Removes and returns the first element from the sorted set stored at key, @@ -871,9 +858,9 @@ public GarnetStatus SortedSetScores(byte[] key, ref ObjectInput /// /// /// - public GarnetStatus SortedSetPop(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetPop(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// Returns the number of elements in the sorted set at key with a score between min and max. @@ -885,8 +872,8 @@ public GarnetStatus SortedSetPop(byte[] key, ref ObjectInput inp /// /// /// - public unsafe GarnetStatus SortedSetCount(ArgSlice key, ArgSlice minScore, ArgSlice maxScore, out int numElements, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public unsafe GarnetStatus SortedSetCount(PinnedSpanByte key, PinnedSpanByte minScore, PinnedSpanByte maxScore, out int numElements, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { numElements = 0; if (key.Length == 0) @@ -901,9 +888,9 @@ public unsafe GarnetStatus SortedSetCount(ArgSlice key, ArgSlice const int outputContainerSize = 32; // 3 for HEADER + CRLF + 20 for ascii long var outputContainer = stackalloc byte[outputContainerSize]; - var output = new GarnetObjectStoreOutput(new(outputContainer, outputContainerSize)); + var output = GarnetObjectStoreOutput.FromPinnedPointer(outputContainer, outputContainerSize); - var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); + var status = ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); if (status == GarnetStatus.OK) { @@ -924,9 +911,9 @@ public unsafe GarnetStatus SortedSetCount(ArgSlice key, ArgSlice /// /// /// - public GarnetStatus SortedSetCount(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Removes all elements in the sorted set between the @@ -938,9 +925,9 @@ public GarnetStatus SortedSetCount(byte[] key, ref ObjectInput i /// /// /// - public GarnetStatus SortedSetRemoveRangeByLex(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperation(key, ref input, out output, ref objectContext); + public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectContext); /// /// Returns the number of elements in the sorted set with a value between min and max. @@ -953,9 +940,9 @@ public GarnetStatus SortedSetRemoveRangeByLex(byte[] key, ref Ob /// /// /// - public GarnetStatus SortedSetLengthByValue(byte[] key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperation(key, ref input, out output, ref objectStoreContext); + public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperation(key.ReadOnlySpan, ref input, out output, ref objectStoreContext); /// /// Increments the score of member in the sorted set stored at key by increment. @@ -967,9 +954,9 @@ public GarnetStatus SortedSetLengthByValue(byte[] key, ref Objec /// /// /// - public GarnetStatus SortedSetIncrement(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetIncrement(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); /// /// ZREMRANGEBYRANK: Removes all elements in the sorted set stored at key with rank between start and stop. @@ -981,9 +968,9 @@ public GarnetStatus SortedSetIncrement(byte[] key, ref ObjectInp /// /// /// - public GarnetStatus SortedSetRemoveRange(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => RMWObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SortedSetRemoveRange(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => RMWObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns the rank of member in the sorted set, the scores in the sorted set are ordered from low to high @@ -994,9 +981,9 @@ public GarnetStatus SortedSetRemoveRange(byte[] key, ref ObjectI /// /// /// - public GarnetStatus SortedSetRank(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SortedSetRank(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Returns a random member from the sorted set key. @@ -1007,9 +994,9 @@ public GarnetStatus SortedSetRank(byte[] key, ref ObjectInput in /// /// /// - public GarnetStatus SortedSetRandomMember(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectContext, ref output); + public GarnetStatus SortedSetRandomMember(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectContext, ref output); /// /// Iterates members of SortedSet key and their associated scores using a cursor, @@ -1021,11 +1008,11 @@ public GarnetStatus SortedSetRandomMember(byte[] key, ref Object /// /// /// - public GarnetStatus SortedSetScan(byte[] key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) - where TObjectContext : ITsavoriteContext - => ReadObjectStoreOperationWithOutput(key, ref input, ref objectStoreContext, ref output); + public GarnetStatus SortedSetScan(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectStoreContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ReadOnlySpan, ref input, ref objectStoreContext, ref output); - public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) { pairs = default; @@ -1043,7 +1030,7 @@ public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights txnManager.Run(true); } - var objectContext = txnManager.ObjectStoreLockableContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; try { @@ -1056,7 +1043,7 @@ public GarnetStatus SortedSetUnion(ReadOnlySpan keys, double[] weights } } - public GarnetStatus SortedSetUnionStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) + public GarnetStatus SortedSetUnionStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) { count = default; @@ -1075,7 +1062,7 @@ public GarnetStatus SortedSetUnionStore(ArgSlice destinationKey, ReadOnlySpan(ReadOnlySpan keys, ref TObjectContext objectContext, + 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; @@ -1125,12 +1111,9 @@ private GarnetStatus SortedSetUnion(ReadOnlySpan keys, return GarnetStatus.OK; // Get the first sorted set - var status = GET(keys[0].ToArray(), out var firstObj, ref objectContext); - + var status = GET(keys[0], out var firstObj, ref objectContext); if (status == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } Dictionary sortedSetDictionary = null; @@ -1163,7 +1146,7 @@ private GarnetStatus SortedSetUnion(ReadOnlySpan keys, // Process remaining sets for (var i = 1; i < keys.Length; i++) { - status = GET(keys[i].ToArray(), out var nextObj, ref objectContext); + status = GET(keys[i], out var nextObj, ref objectContext); if (status == GarnetStatus.WRONGTYPE) return GarnetStatus.WRONGTYPE; if (status != GarnetStatus.OK) @@ -1198,16 +1181,14 @@ private GarnetStatus SortedSetUnion(ReadOnlySpan keys, return GarnetStatus.OK; } - private GarnetStatus SortedSetDifference(ReadOnlySpan keys, ref TObjectContext objectContext, out Dictionary pairs) - where TObjectContext : ITsavoriteContext + private GarnetStatus SortedSetDifference(ReadOnlySpan keys, ref TObjectContext objectContext, out Dictionary pairs) + where TObjectContext : ITsavoriteContext { pairs = default; - var statusOp = GET(keys[0].ToArray(), out var firstObj, ref objectContext); + var statusOp = GET(keys[0], out var firstObj, ref objectContext); if (statusOp == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } if (statusOp == GarnetStatus.NOTFOUND) { @@ -1216,9 +1197,7 @@ private GarnetStatus SortedSetDifference(ReadOnlySpan } if (firstObj.GarnetObject is not SortedSetObject firstSortedSet) - { return GarnetStatus.WRONGTYPE; - } if (keys.Length == 1) { @@ -1229,7 +1208,7 @@ private GarnetStatus SortedSetDifference(ReadOnlySpan // read the rest of the keys for (var item = 1; item < keys.Length; item++) { - statusOp = GET(keys[item].ToArray(), out var nextObj, ref objectContext); + statusOp = GET(keys[item], out var nextObj, ref objectContext); if (statusOp == GarnetStatus.WRONGTYPE) return GarnetStatus.WRONGTYPE; if (statusOp != GarnetStatus.OK) @@ -1253,9 +1232,9 @@ private GarnetStatus SortedSetDifference(ReadOnlySpan /// /// Removes and returns up to count members and their scores from the first sorted set that contains a member. /// - public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out ArgSlice poppedKey, out (ArgSlice member, ArgSlice score)[] pairs) + public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, bool lowScoresFirst, out PinnedSpanByte poppedKey, out (PinnedSpanByte member, PinnedSpanByte score)[] pairs) { - if (txnManager.ObjectStoreLockableContext.Session is null) + if (txnManager.ObjectStoreTransactionalContext.Session is null) ThrowObjectStoreUninitializedException(); pairs = default; @@ -1275,7 +1254,7 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, txnManager.Run(true); } - var storeLockableContext = txnManager.ObjectStoreLockableContext; + var storeTransactionalContext = txnManager.ObjectStoreTransactionalContext; try { @@ -1284,7 +1263,7 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, { if (key.Length == 0) continue; - var status = SortedSetPop(key, count, lowScoresFirst, out pairs, ref storeLockableContext); + var status = SortedSetPop(key, count, lowScoresFirst, out pairs, ref storeTransactionalContext); if (status == GarnetStatus.OK && pairs != null && pairs.Length > 0) { poppedKey = key; @@ -1292,9 +1271,7 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, } if (status != GarnetStatus.OK && status != GarnetStatus.NOTFOUND) - { return status; - } } return GarnetStatus.OK; @@ -1309,7 +1286,7 @@ public unsafe GarnetStatus SortedSetMPop(ReadOnlySpan keys, int count, /// /// Computes the cardinality of the intersection of multiple sorted sets. /// - public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) + public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? limit, out int count) { count = 0; @@ -1325,7 +1302,7 @@ public GarnetStatus SortedSetIntersectLength(ReadOnlySpan keys, int? l /// /// Computes the intersection of multiple sorted sets and stores the resulting sorted set at destinationKey. /// - public GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) + public GarnetStatus SortedSetIntersectStore(PinnedSpanByte destinationKey, ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out int count) { count = default; @@ -1344,7 +1321,7 @@ public GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpa _ = txnManager.Run(true); } - var objectContext = txnManager.ObjectStoreLockableContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; try { @@ -1365,14 +1342,13 @@ public GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpa newSortedSetObject.Add(element, score); } - var destinationKeyBytes = destinationKey.ToArray(); - _ = SET(destinationKeyBytes, newSortedSetObject, ref objectContext); - itemBroker.HandleCollectionUpdate(destinationKeyBytes); + _ = SET(destinationKey, newSortedSetObject, ref objectContext); + itemBroker.HandleCollectionUpdate(destinationKey.ToArray()); } else { _ = EXPIRE(destinationKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None, - ref lockableContext, ref objectContext); + ref transactionalContext, ref objectContext); } return status; @@ -1387,7 +1363,7 @@ public GarnetStatus SortedSetIntersectStore(ArgSlice destinationKey, ReadOnlySpa /// /// Computes the intersection of multiple sorted sets and returns the result with optional weights and aggregate type. /// - public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) + public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, out Dictionary pairs) { pairs = default; @@ -1405,7 +1381,7 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] wei txnManager.Run(true); } - var objectContext = txnManager.ObjectStoreLockableContext; + var objectContext = txnManager.ObjectStoreTransactionalContext; try { @@ -1428,17 +1404,14 @@ public GarnetStatus SortedSetIntersect(ReadOnlySpan keys, double[] wei /// The object context. /// 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 + private GarnetStatus SortedSetIntersection(ReadOnlySpan keys, double[] weights, SortedSetAggregateType aggregateType, ref TObjectContext objectContext, out Dictionary pairs) + where TObjectContext : ITsavoriteContext { pairs = default; - var statusOp = GET(keys[0].ToArray(), out var firstObj, ref objectContext); - + var statusOp = GET(keys[0], out var firstObj, ref objectContext); if (statusOp == GarnetStatus.WRONGTYPE) - { return GarnetStatus.WRONGTYPE; - } if (statusOp == GarnetStatus.NOTFOUND) { @@ -1466,14 +1439,12 @@ private GarnetStatus SortedSetIntersection(ReadOnlySpan(ReadOnlySpanThe type of the object context. /// The key for which to set the expiration time. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus SortedSetExpire(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetExpire(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { return RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); } @@ -1542,8 +1513,8 @@ public GarnetStatus SortedSetExpire(ArgSlice key, ref ObjectInpu /// The results of the operation, indicating the number of fields that were successfully set to expire. /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. - public GarnetStatus SortedSetExpire(ArgSlice key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetExpire(PinnedSpanByte key, ReadOnlySpan members, DateTimeOffset expireAt, ExpireOption expireOption, out int[] results, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { results = default; var expireMillisecs = expireAt.ToUnixTimeMilliseconds(); @@ -1577,14 +1548,12 @@ public GarnetStatus SortedSetExpire(ArgSlice key, ReadOnlySpanThe type of the object context. /// The key of the hash. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext - { - return ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); - } + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext + => ReadObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); /// /// Returns the time-to-live (TTL) of a SortedSet member. @@ -1595,8 +1564,8 @@ public GarnetStatus SortedSetTimeToLive(ArgSlice key, ref Object /// The array of TimeSpan representing the TTL for each member. /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. - public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySpan members, out TimeSpan[] expireIn, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetTimeToLive(PinnedSpanByte key, ReadOnlySpan members, out TimeSpan[] expireIn, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { expireIn = default; var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZTTL }; @@ -1610,10 +1579,7 @@ public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySp var status = ReadObjectStoreOperationWithOutput(key.ToArray(), ref innerInput, ref objectContext, ref output); if (status == GarnetStatus.OK) - { - expireIn = ProcessRespInt64ArrayOutput(output, out _).Select(x => TimeSpan.FromMilliseconds(x < 0 ? 0 : x)).ToArray(); - } - + expireIn = [.. ProcessRespInt64ArrayOutput(output, out _).Select(x => TimeSpan.FromMilliseconds(x < 0 ? 0 : x))]; return status; } @@ -1623,11 +1589,11 @@ public GarnetStatus SortedSetTimeToLive(ArgSlice key, ReadOnlySp /// The type of the object context. /// The key of the SortedSet. /// The input object containing the operation details. - /// The output footer object to store the result. + /// The output object to store the result. /// The object context for the operation. /// The status of the operation. - public GarnetStatus SortedSetPersist(ArgSlice key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetPersist(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext => RMWObjectStoreOperationWithOutput(key.ToArray(), ref input, ref objectContext, ref output); /// @@ -1639,8 +1605,8 @@ public GarnetStatus SortedSetPersist(ArgSlice key, ref ObjectInp /// The results of the operation, indicating the number of members whose expiration time was successfully removed. /// The context of the object store. /// Returns a GarnetStatus indicating the success or failure of the operation. - public GarnetStatus SortedSetPersist(ArgSlice key, ReadOnlySpan members, out int[] results, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetPersist(PinnedSpanByte key, ReadOnlySpan members, out int[] results, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { results = default; var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZPERSIST }; @@ -1671,8 +1637,8 @@ public GarnetStatus SortedSetPersist(ArgSlice key, ReadOnlySpan< /// If the first key is "*", all SortedSet keys are scanned in batches and the operation is performed on each key. /// Otherwise, the operation is performed on the specified keys. /// - public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref ObjectInput input, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { if (keys[0].ReadOnlySpan.SequenceEqual("*"u8)) { @@ -1698,7 +1664,7 @@ public GarnetStatus SortedSetCollect(ReadOnlySpan keys /// Otherwise, the operation is performed on the specified keys. /// public GarnetStatus SortedSetCollect(ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + where TObjectContext : ITsavoriteContext { return SortedSetCollect([], ref objectContext); } @@ -1714,15 +1680,15 @@ public GarnetStatus SortedSetCollect(ref TObjectContext objectCo /// If the first key is "*", all SortedSet keys are scanned in batches and the operation is performed on each key. /// Otherwise, the operation is performed on the specified keys. /// - public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref TObjectContext objectContext) - where TObjectContext : ITsavoriteContext + public GarnetStatus SortedSetCollect(ReadOnlySpan keys, ref TObjectContext objectContext) + where TObjectContext : ITsavoriteContext { var header = new RespInputHeader(GarnetObjectType.SortedSet) { SortedSetOp = SortedSetOperation.ZCOLLECT }; var innerInput = new ObjectInput(header); if (keys.IsEmpty) { - return SortedSetCollect([ArgSlice.FromPinnedSpan("*"u8)], ref innerInput, ref objectContext); + return SortedSetCollect([PinnedSpanByte.FromPinnedSpan("*"u8)], ref innerInput, ref objectContext); } return SortedSetCollect(keys, ref innerInput, ref objectContext); diff --git a/libs/server/Storage/Session/StorageSession.cs b/libs/server/Storage/Session/StorageSession.cs index c08f094ca92..71154fd98ff 100644 --- a/libs/server/Storage/Session/StorageSession.cs +++ b/libs/server/Storage/Session/StorageSession.cs @@ -8,11 +8,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Storage Session - the internal layer that Garnet uses to perform storage operations @@ -26,8 +26,8 @@ sealed partial class StorageSession : IDisposable /// /// Session Contexts for main store /// - public BasicContext basicContext; - public LockableContext lockableContext; + public BasicContext basicContext; + public TransactionalContext transactionalContext; SectorAlignedMemory sectorAlignedMemoryHll1; SectorAlignedMemory sectorAlignedMemoryHll2; @@ -39,8 +39,8 @@ sealed partial class StorageSession : IDisposable /// /// Session Contexts for object store /// - public BasicContext objectStoreBasicContext; - public LockableContext objectStoreLockableContext; + public BasicContext objectStoreBasicContext; + public TransactionalContext objectStoreTransactionalContext; public readonly ScratchBufferManager scratchBufferManager; public readonly FunctionsState functionsState; @@ -84,11 +84,11 @@ public StorageSession(StoreWrapper storeWrapper, var objectStoreSession = db.ObjectStore?.NewSession(objectStoreFunctions); basicContext = session.BasicContext; - lockableContext = session.LockableContext; + transactionalContext = session.TransactionalContext; if (objectStoreSession != null) { objectStoreBasicContext = objectStoreSession.BasicContext; - objectStoreLockableContext = objectStoreSession.LockableContext; + objectStoreTransactionalContext = objectStoreSession.TransactionalContext; } HeadAddress = db.MainStore.Log.HeadAddress; diff --git a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs index 9c7f63527f0..4adf5de8746 100644 --- a/libs/server/Storage/SizeTracker/CacheSizeTracker.cs +++ b/libs/server/Storage/SizeTracker/CacheSizeTracker.cs @@ -10,8 +10,8 @@ namespace Garnet.server { - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = 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; public long TargetSize; public long ReadCacheTargetSize; @@ -31,21 +31,23 @@ public class CacheSizeTracker internal bool Stopped => (mainLogTracker == null || mainLogTracker.Stopped) && (readCacheTracker == null || readCacheTracker.Stopped); /// Helps calculate size of a record including heap memory in Object store. - internal struct LogSizeCalculator : ILogSizeCalculator + internal struct LogSizeCalculator : ILogSizeCalculator { /// Calculate the size of a record in the cache - /// Information about the record - /// The record's key - /// The record's value + /// Information about the record /// The size of the record [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long CalculateRecordSize(RecordInfo recordInfo, byte[] key, IGarnetObject value) + public readonly long CalculateRecordSize(ref TSourceLogRecord logRecord) + where TSourceLogRecord : ISourceLogRecord { - long size = Utility.RoundUp(key.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead; - - if (!recordInfo.Tombstone && value != null) // ignore deleted values being evicted (they are accounted for by ConcurrentDeleter) - size += value.Size; - + long size = Utility.RoundUp(logRecord.Key.Length, IntPtr.Size) + MemoryUtils.ByteArrayOverhead; + + if (!logRecord.Info.Tombstone) + { + var value = logRecord.ValueObject; + if (value != null) // ignore deleted values being evicted (they are accounted for by InPlaceDeleter) + size += value.MemorySize; + } return size; } } @@ -56,7 +58,7 @@ public readonly long CalculateRecordSize(RecordInfo recordInfo, byte[] key, IGar /// 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); @@ -69,19 +71,19 @@ public CacheSizeTracker(TsavoriteKV 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 39faa941627..42aa4871955 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -18,11 +18,11 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Wrapper for store and store-specific information @@ -42,12 +42,12 @@ public sealed class StoreWrapper /// /// Store (of DB 0) /// - public TsavoriteKV store => databaseManager.MainStore; + public TsavoriteKV store => databaseManager.MainStore; /// /// Object store (of DB 0) /// - public TsavoriteKV objectStore => databaseManager.ObjectStore; + public TsavoriteKV objectStore => databaseManager.ObjectStore; /// /// AOF (of DB 0) @@ -704,7 +704,7 @@ static void ExecuteHashCollect(ScratchBufferManager scratchBufferManager, Storag var header = new RespInputHeader(GarnetObjectType.Hash) { HashOp = HashOperation.HCOLLECT }; var input = new ObjectInput(header); - ReadOnlySpan key = [ArgSlice.FromPinnedSpan("*"u8)]; + ReadOnlySpan key = [PinnedSpanByte.FromPinnedSpan("*"u8)]; storageSession.HashCollect(key, ref input, ref storageSession.objectStoreBasicContext); scratchBufferManager.Reset(); } @@ -780,15 +780,13 @@ public bool HasKeysInSlots(List slots) { bool hasKeyInSlots = false; { - using var iter = store.Iterate>(new SimpleSessionFunctions()); - while (!hasKeyInSlots && iter.GetNext(out RecordInfo record)) + using var iter = store.Iterate(new SimpleGarnetObjectSessionFunctions()); // TODO replace with Push iterator + while (!hasKeyInSlots && iter.GetNext()) { - ref var key = ref iter.GetKey(); - ushort hashSlotForKey = HashSlotUtils.HashSlot(ref key); + var key = iter.Key; + ushort hashSlotForKey = HashSlotUtils.HashSlot(key); if (slots.Contains(hashSlotForKey)) - { hasKeyInSlots = true; - } } } @@ -798,14 +796,11 @@ public bool HasKeysInSlots(List slots) var objstorefunctions = new ObjectSessionFunctions(functionsState); var objectStoreSession = objectStore?.NewSession(objstorefunctions); var iter = objectStoreSession.Iterate(); - while (!hasKeyInSlots && iter.GetNext(out RecordInfo record)) + while (!hasKeyInSlots && iter.GetNext()) { - ref var key = ref iter.GetKey(); - ushort hashSlotForKey = HashSlotUtils.HashSlot(key.AsSpan()); + ushort hashSlotForKey = HashSlotUtils.HashSlot(iter.Key); if (slots.Contains(hashSlotForKey)) - { hasKeyInSlots = true; - } } } diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 05f4f8843a3..7992bd7e717 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -10,24 +10,24 @@ namespace Garnet.server { - using BasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; - using LockableGarnetApi = GarnetApi, - SpanByteAllocator>>, - LockableContext>, - GenericAllocator>>>>; - - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; - - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using BasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; + using TransactionalGarnetApi = GarnetApi, + SpanByteAllocator>>, + TransactionalContext, + ObjectAllocator>>>; + + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; + + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Transaction manager @@ -37,28 +37,28 @@ public sealed unsafe partial class TransactionManager /// /// Basic context for main store /// - readonly BasicContext basicContext; + readonly BasicContext basicContext; /// - /// Lockable context for main store + /// Transactional context for main store /// - readonly LockableContext lockableContext; + readonly TransactionalContext transactionalContext; /// /// Basic context for object store /// - readonly BasicContext objectStoreBasicContext; + readonly BasicContext objectStoreBasicContext; /// - /// Lockable context for object store + /// Transactional context for object store /// - readonly LockableContext objectStoreLockableContext; + readonly TransactionalContext objectStoreTransactionalContext; // Not readonly to avoid defensive copy GarnetWatchApi garnetTxPrepareApi; // Not readonly to avoid defensive copy - LockableGarnetApi garnetTxMainApi; + TransactionalGarnetApi garnetTxMainApi; // Not readonly to avoid defensive copy BasicGarnetApi garnetTxFinalizeApi; @@ -82,12 +82,12 @@ public sealed unsafe partial class TransactionManager readonly ILogger logger; long txnVersion; - internal LockableContext LockableContext - => lockableContext; - internal LockableUnsafeContext LockableUnsafeContext - => basicContext.Session.LockableUnsafeContext; - internal LockableContext ObjectStoreLockableContext - => objectStoreLockableContext; + internal TransactionalContext TransactionalContext + => transactionalContext; + internal TransactionalUnsafeContext TransactionalUnsafeContext + => basicContext.Session.TransactionalUnsafeContext; + internal TransactionalContext ObjectStoreTransactionalContext + => objectStoreTransactionalContext; /// /// Array to keep pointer keys in keyBuffer @@ -97,7 +97,7 @@ internal LockableContext(garnetApi); garnetTxFinalizeApi = garnetApi; this.clusterEnabled = clusterEnabled; if (clusterEnabled) - keys = new ArgSlice[initialKeyBufferSize]; + keys = new PinnedSpanByte[initialKeyBufferSize]; Reset(false); } @@ -150,12 +150,12 @@ internal void Reset(bool isRunning) // Release context if (transactionStoreType == StoreType.Main || transactionStoreType == StoreType.All) - lockableContext.EndLockable(); + 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"); - objectStoreLockableContext.EndLockable(); + objectStoreTransactionalContext.EndTransaction(); } } finally @@ -269,7 +269,7 @@ internal void Commit(bool internal_txn = false) Reset(true); } - internal void Watch(ArgSlice key, StoreType type) + internal void Watch(PinnedSpanByte key, StoreType type) { // Update watch type if object store is disabled if (type == StoreType.All && objectStoreBasicContext.IsNull) @@ -279,9 +279,9 @@ internal void Watch(ArgSlice key, StoreType type) watchContainer.AddWatch(key, type); if (type == StoreType.Main || type == StoreType.All) - basicContext.ResetModified(key.SpanByte); + basicContext.ResetModified(key.ReadOnlySpan); if ((type == StoreType.Object || type == StoreType.All) && !objectStoreBasicContext.IsNull) - objectStoreBasicContext.ResetModified(key.ToArray()); + objectStoreBasicContext.ResetModified(key.ReadOnlySpan); } void UpdateTransactionStoreType(StoreType type) @@ -300,7 +300,7 @@ void UpdateTransactionStoreType(StoreType type) internal string GetLockset() => keyEntries.GetLockset(); - internal void GetKeysForValidation(byte* recvBufferPtr, out ArgSlice[] keys, out int keyCount, out bool readOnly) + internal void GetKeysForValidation(byte* recvBufferPtr, out PinnedSpanByte[] keys, out int keyCount, out bool readOnly) { UpdateRecvBufferPtr(recvBufferPtr); watchContainer.SaveKeysToKeyList(this); @@ -309,17 +309,17 @@ internal void GetKeysForValidation(byte* recvBufferPtr, out ArgSlice[] keys, out readOnly = keyEntries.IsReadOnly; } - void BeginLockable(StoreType transactionStoreType) + void BeginTransaction(StoreType transactionStoreType) { if (transactionStoreType is StoreType.All or StoreType.Main) { - lockableContext.BeginLockable(); + 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"); - objectStoreLockableContext.BeginLockable(); + objectStoreTransactionalContext.BeginTransaction(); } } @@ -327,13 +327,13 @@ void LocksAcquired(StoreType transactionStoreType, long txnVersion) { if (transactionStoreType is StoreType.All or StoreType.Main) { - lockableContext.LocksAcquired(txnVersion); + 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"); - objectStoreLockableContext.LocksAcquired(txnVersion); + objectStoreTransactionalContext.LocksAcquired(txnVersion); } } @@ -347,7 +347,7 @@ internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, Tim txnVersion = stateMachineDriver.AcquireTransactionVersion(); // Acquire lock sessions - BeginLockable(transactionStoreType); + BeginTransaction(transactionStoreType); bool lockSuccess; if (fail_fast_on_lock) diff --git a/libs/server/Transaction/TxnClusterSlotCheck.cs b/libs/server/Transaction/TxnClusterSlotCheck.cs index da587fc774f..1d17a1dee38 100644 --- a/libs/server/Transaction/TxnClusterSlotCheck.cs +++ b/libs/server/Transaction/TxnClusterSlotCheck.cs @@ -2,13 +2,14 @@ // Licensed under the MIT license. using System; +using Tsavorite.core; namespace Garnet.server { sealed unsafe partial class TransactionManager { // Keys involved in the current transaction - ArgSlice[] keys; + PinnedSpanByte[] keys; int keyCount; internal byte* saveKeyRecvBufferPtr; @@ -18,7 +19,7 @@ sealed unsafe partial class TransactionManager /// Keep track of actual key accessed by command /// /// - public void SaveKeyArgSlice(ArgSlice argSlice) + public void SaveKeyArgSlice(PinnedSpanByte argSlice) { // Execute method only if clusterEnabled if (!clusterEnabled) return; @@ -26,7 +27,7 @@ public void SaveKeyArgSlice(ArgSlice argSlice) if (keyCount >= keys.Length) { var oldKeys = keys; - keys = new ArgSlice[keys.Length * 2]; + keys = new PinnedSpanByte[keys.Length * 2]; Array.Copy(oldKeys, keys, oldKeys.Length); } diff --git a/libs/server/Transaction/TxnKeyEntry.cs b/libs/server/Transaction/TxnKeyEntry.cs index a121d603df2..6ea3bd20dc5 100644 --- a/libs/server/Transaction/TxnKeyEntry.cs +++ b/libs/server/Transaction/TxnKeyEntry.cs @@ -8,17 +8,17 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; /// /// Entry for a key to lock and unlock in transactions /// [StructLayout(LayoutKind.Explicit, Size = 10)] - struct TxnKeyEntry : ILockableKey + struct TxnKeyEntry : ITransactionalKey { [FieldOffset(0)] internal long keyHash; @@ -29,20 +29,20 @@ struct TxnKeyEntry : ILockableKey [FieldOffset(9)] internal LockType lockType; - #region ILockableKey + #region ITransactionalKey /// - public long KeyHash { get => keyHash; } + public readonly long KeyHash { get => keyHash; } /// - public LockType LockType { get => lockType; } - #endregion ILockableKey + public readonly LockType LockType { get => lockType; } + #endregion ITransactionalKey /// - public override string ToString() + 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 = this.keyHash >= 0 ? this.keyHash : -this.keyHash; + var absKeyHash = keyHash >= 0 ? keyHash : -keyHash; return $"{keyHashSign}{absKeyHash}:{(isObject ? "obj" : "raw")}:{(lockType == LockType.None ? "-" : (lockType == LockType.Shared ? "s" : "x"))}"; } } @@ -61,19 +61,19 @@ internal sealed class TxnKeyEntries public int phase; - internal TxnKeyEntries(int initialCount, LockableContext lockableContext, - LockableContext objectStoreLockableContext) + 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, - comparison = new(lockableContext, objectStoreLockableContext); + comparison = new(transactionalContext, objectStoreTransactionalContext); } public bool IsReadOnly { get { - bool readOnly = true; + var readOnly = true; for (int i = 0; i < keyCount; i++) { if (keys[i].lockType == LockType.Exclusive) @@ -86,11 +86,11 @@ public bool IsReadOnly } } - public void AddKey(ArgSlice keyArgSlice, bool isObject, LockType type) + public void AddKey(PinnedSpanByte keyArgSlice, bool isObject, LockType type) { var keyHash = !isObject - ? comparison.lockableContext.GetKeyHash(keyArgSlice.SpanByte) - : comparison.objectStoreLockableContext.GetKeyHash(keyArgSlice.ToArray()); + ? comparison.transactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan) + : comparison.objectStoreTransactionalContext.GetKeyHash(keyArgSlice.ReadOnlySpan); // Grow the buffer if needed if (keyCount >= keys.Length) @@ -122,14 +122,14 @@ internal void LockAllKeys() // Issue main store locks if (mainKeyCount > 0) { - comparison.lockableContext.Lock(keys, 0, mainKeyCount); + comparison.transactionalContext.Lock(keys, 0, mainKeyCount); mainStoreKeyLocked = true; } // Issue object store locks if (mainKeyCount < keyCount) { - comparison.objectStoreLockableContext.Lock(keys, mainKeyCount, keyCount - mainKeyCount); + comparison.objectStoreTransactionalContext.Lock(keys, mainKeyCount, keyCount - mainKeyCount); objectStoreKeyLocked = true; } @@ -150,7 +150,7 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) // TryLock will unlock automatically in case of partial failure if (mainKeyCount > 0) { - mainStoreKeyLocked = comparison.lockableContext.TryLock(keys, 0, mainKeyCount, lock_timeout); + mainStoreKeyLocked = comparison.transactionalContext.TryLock(keys, 0, mainKeyCount, lock_timeout); if (!mainStoreKeyLocked) { phase = 0; @@ -162,7 +162,7 @@ internal bool TryLockAllKeys(TimeSpan lock_timeout) // TryLock will unlock automatically in case of partial failure if (mainKeyCount < keyCount) { - objectStoreKeyLocked = comparison.objectStoreLockableContext.TryLock(keys, mainKeyCount, keyCount - mainKeyCount, lock_timeout); + objectStoreKeyLocked = comparison.objectStoreTransactionalContext.TryLock(keys, mainKeyCount, keyCount - mainKeyCount, lock_timeout); if (!objectStoreKeyLocked) { phase = 0; @@ -178,9 +178,9 @@ internal void UnlockAllKeys() { phase = 2; if (mainStoreKeyLocked && mainKeyCount > 0) - comparison.lockableContext.Unlock(keys, 0, mainKeyCount); + comparison.transactionalContext.Unlock(keys, 0, mainKeyCount); if (objectStoreKeyLocked && mainKeyCount < keyCount) - comparison.objectStoreLockableContext.Unlock(keys, mainKeyCount, keyCount - mainKeyCount); + comparison.objectStoreTransactionalContext.Unlock(keys, mainKeyCount, keyCount - mainKeyCount); mainKeyCount = 0; keyCount = 0; mainStoreKeyLocked = false; @@ -196,12 +196,12 @@ internal string GetLockset() for (int ii = 0; ii < keyCount; ii++) { ref var entry = ref keys[ii]; - sb.Append(delimiter); - sb.Append(entry.ToString()); + _ = sb.Append(delimiter); + _ = sb.Append(entry.ToString()); } if (sb.Length > 0) - sb.Append($" (phase: {(phase == 0 ? "none" : (phase == 1 ? "lock" : "unlock"))}))"); + _ = sb.Append($" (phase: {(phase == 0 ? "none" : (phase == 1 ? "lock" : "unlock"))}))"); return sb.ToString(); } } diff --git a/libs/server/Transaction/TxnKeyEntryComparison.cs b/libs/server/Transaction/TxnKeyEntryComparison.cs index 8ab74b1f212..bc2df1ffd29 100644 --- a/libs/server/Transaction/TxnKeyEntryComparison.cs +++ b/libs/server/Transaction/TxnKeyEntryComparison.cs @@ -6,24 +6,24 @@ namespace Garnet.server { - using MainStoreAllocator = SpanByteAllocator>; - using MainStoreFunctions = StoreFunctions; + using MainStoreAllocator = SpanByteAllocator>; + using MainStoreFunctions = StoreFunctions; - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; internal sealed class TxnKeyComparison { - public LockableContext lockableContext; - public LockableContext objectStoreLockableContext; + public TransactionalContext transactionalContext; + public TransactionalContext objectStoreTransactionalContext; public readonly Comparison comparisonDelegate; - internal TxnKeyComparison(LockableContext lockableContext, - LockableContext objectStoreLockableContext) + internal TxnKeyComparison(TransactionalContext transactionalContext, + TransactionalContext objectStoreTransactionalContext) { - this.lockableContext = lockableContext; - this.objectStoreLockableContext = objectStoreLockableContext; + this.transactionalContext = transactionalContext; + this.objectStoreTransactionalContext = objectStoreTransactionalContext; comparisonDelegate = Compare; } @@ -35,9 +35,9 @@ public int Compare(TxnKeyEntry key1, TxnKeyEntry key2) if (cmp != 0) return cmp; if (key1.isObject) - return objectStoreLockableContext.CompareKeyHashes(ref key1, ref key2); + return objectStoreTransactionalContext.CompareKeyHashes(ref key1, ref key2); else - return lockableContext.CompareKeyHashes(ref key1, ref key2); + return transactionalContext.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 0c08171c592..95e7cd8783c 100644 --- a/libs/server/Transaction/TxnKeyManager.cs +++ b/libs/server/Transaction/TxnKeyManager.cs @@ -15,7 +15,7 @@ sealed partial class TransactionManager /// /// /// - public void SaveKeyEntryToLock(ArgSlice key, bool isObject, LockType type) + public void SaveKeyEntryToLock(PinnedSpanByte key, bool isObject, LockType type) { UpdateTransactionStoreType(isObject ? StoreType.Object : StoreType.Main); keyEntries.AddKey(key, isObject, type); @@ -44,7 +44,7 @@ public void WriteCachedSlotVerificationMessage(ref MemoryResult output) /// /// /// - public unsafe void VerifyKeyOwnership(ArgSlice key, LockType type) + public unsafe void VerifyKeyOwnership(PinnedSpanByte key, LockType type) { if (!clusterEnabled) return; diff --git a/libs/server/Transaction/TxnRespCommands.cs b/libs/server/Transaction/TxnRespCommands.cs index c746c91f0cd..0d3a3931144 100644 --- a/libs/server/Transaction/TxnRespCommands.cs +++ b/libs/server/Transaction/TxnRespCommands.cs @@ -5,6 +5,7 @@ using System.Collections.Generic; using Garnet.common; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.server { @@ -224,7 +225,7 @@ private bool CommonWATCH(StoreType type) return true; } - List keys = []; + List keys = []; for (var c = 0; c < count; c++) { diff --git a/libs/server/Transaction/TxnWatchedKeysContainer.cs b/libs/server/Transaction/TxnWatchedKeysContainer.cs index 6907ed32e6d..32fcd6ca387 100644 --- a/libs/server/Transaction/TxnWatchedKeysContainer.cs +++ b/libs/server/Transaction/TxnWatchedKeysContainer.cs @@ -49,7 +49,7 @@ public void Reset() watchBufferHeadAddress = 0; } - public bool RemoveWatch(ArgSlice key) + public bool RemoveWatch(PinnedSpanByte key) { for (int i = 0; i < sliceCount; i++) { @@ -62,7 +62,7 @@ public bool RemoveWatch(ArgSlice key) return false; } - public void AddWatch(ArgSlice key, StoreType type) + public void AddWatch(PinnedSpanByte key, StoreType type) { if (sliceCount >= sliceBufferSize) { @@ -92,12 +92,12 @@ public void AddWatch(ArgSlice key, StoreType type) } } - var slice = new ArgSlice(watchBufferPtr, key.Length); - key.ReadOnlySpan.CopyTo(slice.Span); + var slice = PinnedSpanByte.FromPinnedPointer(watchBufferPtr, key.Length); + key. ReadOnlySpan.CopyTo(slice.Span); keySlices[sliceCount].slice = slice; keySlices[sliceCount].type = type; - keySlices[sliceCount].hash = Utility.HashBytes(slice.ptr, slice.Length); + keySlices[sliceCount].hash = Utility.HashBytes(slice.ReadOnlySpan); keySlices[sliceCount].version = versionMap.ReadVersion(keySlices[sliceCount].hash); watchBufferPtr += key.Length; diff --git a/libs/server/Transaction/WatchedKeySlice.cs b/libs/server/Transaction/WatchedKeySlice.cs index 6af999e6669..42c393b2c96 100644 --- a/libs/server/Transaction/WatchedKeySlice.cs +++ b/libs/server/Transaction/WatchedKeySlice.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System.Runtime.InteropServices; +using Tsavorite.core; namespace Garnet.server { @@ -12,7 +13,7 @@ struct WatchedKeySlice public long version; [FieldOffset(8)] - public ArgSlice slice; + public PinnedSpanByte slice; [FieldOffset(20)] public long hash; 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 9efa88bdb48..a74813e2145 100644 --- a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs +++ b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/InliningTests.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. +using System.Runtime.InteropServices; using BenchmarkDotNet.Attributes; using BenchmarkDotNet.Configs; using BenchmarkDotNet.Diagnostics.Windows.Configs; @@ -14,7 +15,7 @@ namespace BenchmarkDotNetTests { #pragma warning disable IDE0065 // Misplaced using directive - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; [InliningDiagnoser(logFailuresOnly: true, allowedNamespaces: ["Tsavorite.core"])] [GroupBenchmarksBy(BenchmarkLogicalGroupRule.ByCategory, BenchmarkLogicalGroupRule.ByParams)] @@ -23,7 +24,7 @@ public class InliningTests [Params(1_000_000)] public int NumRecords; - TsavoriteKV> store; + TsavoriteKV> store; IDevice logDevice; string logDirectory; @@ -37,27 +38,24 @@ void SetupStore() { IndexSize = 1L << 26, LogDevice = logDevice - }, StoreFunctions.Create() + }, StoreFunctions.Create() , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } unsafe void PopulateStore() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var keySpanByte = SpanByte.FromPinnedSpan(keyVec); - - Span valueVec = stackalloc byte[sizeof(long)]; - var valueSpanByte = SpanByte.FromPinnedSpan(valueVec); + Span keySpan = stackalloc byte[sizeof(long)]; + Span valueSpan = stackalloc byte[sizeof(long)]; for (long ii = 0; ii < NumRecords; ++ii) { - *(long*)keySpanByte.ToPointer() = ii; - *(long*)valueSpanByte.ToPointer() = ii + NumRecords; - _ = bContext.Upsert(keySpanByte, valueSpanByte); + MemoryMarshal.Cast(keySpan)[0] = ii; + MemoryMarshal.Cast(valueSpan)[0] = ii + NumRecords; + _ = bContext.Upsert(keySpan, valueSpan); } } @@ -85,40 +83,35 @@ public void TearDown() [BenchmarkCategory("Upsert"), Benchmark] public unsafe void Upsert() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var keySpanByte = SpanByte.FromPinnedSpan(keyVec); - - Span valueVec = stackalloc byte[sizeof(long)]; - var valueSpanByte = SpanByte.FromPinnedSpan(valueVec); + Span keySpan = stackalloc byte[sizeof(long)]; + Span valueSpan = stackalloc byte[sizeof(long)]; for (long ii = 0; ii < NumRecords; ++ii) { - *(long*)keySpanByte.ToPointer() = ii; - *(long*)valueSpanByte.ToPointer() = ii + NumRecords * 2; - _ = bContext.Upsert(keySpanByte, valueSpanByte); + MemoryMarshal.Cast(keySpan)[0] = ii; + MemoryMarshal.Cast(valueSpan)[0] = ii + NumRecords * 2; + _ = bContext.Upsert(keySpan, valueSpan); } } [BenchmarkCategory("RMW"), Benchmark] public unsafe void RMW() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var keySpanByte = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[sizeof(long)]; - var inputSpanByte = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[sizeof(long)]; + Span input = stackalloc byte[sizeof(long)]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); for (long ii = 0; ii < NumRecords; ++ii) { - *(long*)keySpanByte.ToPointer() = ii; - *(long*)inputSpanByte.ToPointer() = ii + NumRecords * 3; - _ = bContext.RMW(keySpanByte, inputSpanByte); + MemoryMarshal.Cast(key)[0] = ii; + MemoryMarshal.Cast(input)[0] = ii + NumRecords * 3; + _ = bContext.RMW(key, ref pinnedInputSpan); } _ = bContext.CompletePending(); @@ -127,16 +120,15 @@ public unsafe void RMW() [BenchmarkCategory("Read"), Benchmark] public unsafe void Read() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var keySpanByte = SpanByte.FromPinnedSpan(keyVec); + Span keySpan = stackalloc byte[sizeof(long)]; for (long ii = 0; ii < NumRecords; ++ii) { - *(long*)keySpanByte.ToPointer() = ii; - _ = bContext.Read(keySpanByte); + MemoryMarshal.Cast(keySpan)[0] = ii; + _ = bContext.Read(keySpan); } _ = bContext.CompletePending(); } 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 519c32c55cb..2eac2707605 100644 --- a/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs +++ b/libs/storage/Tsavorite/cs/benchmark/BDN-Tsavorite.Benchmark/IterationTests.cs @@ -13,7 +13,7 @@ namespace BenchmarkDotNetTests { #pragma warning disable IDE0065 // Misplaced using directive - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; [GroupBenchmarksBy(BenchmarkLogicalGroupRule.ByCategory, BenchmarkLogicalGroupRule.ByParams)] public class IterationTests @@ -23,7 +23,7 @@ public class IterationTests [Params(true, false)] public bool FlushAndEvict; - TsavoriteKV> store; + TsavoriteKV> store; IDevice logDevice; string logDirectory; @@ -37,27 +37,25 @@ void SetupStore() { IndexSize = 1L << 26, LogDevice = logDevice - }, StoreFunctions.Create() + }, StoreFunctions.Create() , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } unsafe void PopulateStore() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var keySpanByte = SpanByte.FromPinnedSpan(keyVec); - - Span valueVec = stackalloc byte[sizeof(long)]; - var valueSpanByte = SpanByte.FromPinnedSpan(valueVec); + long keyNum = 0, valueNum = 0; + Span key = SpanByte.FromPinnedVariable(ref keyNum); + Span value = SpanByte.FromPinnedVariable(ref valueNum); for (long ii = 0; ii < NumRecords; ++ii) { - *(long*)keySpanByte.ToPointer() = ii; - *(long*)valueSpanByte.ToPointer() = ii + NumRecords; - _ = bContext.Upsert(keySpanByte, valueSpanByte); + keyNum = ii; + valueNum = ii + NumRecords; + _ = bContext.Upsert(key, value); } if (FlushAndEvict) @@ -88,7 +86,7 @@ public void TearDown() [BenchmarkCategory("Cursor"), Benchmark] public void Cursor() { - using var session = store.NewSession>(new()); + using var session = store.NewSession>(new()); var scanFunctions = new ScanFunctions(); var cursor = 0L; @@ -102,7 +100,7 @@ class ScanCounter internal int count; } - internal struct ScanFunctions : IScanIteratorFunctions + internal struct ScanFunctions : IScanIteratorFunctions { private readonly ScanCounter counter; @@ -114,17 +112,14 @@ internal struct ScanFunctions : IScanIteratorFunctions public bool OnStart(long beginAddress, long endAddress) => true; /// - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { ++counter.count; cursorRecordResult = CursorRecordResult.Accept; return true; } - /// - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - /// public void OnStop(bool completed, long numberOfRecords) { } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ConcurrentDictionaryBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ConcurrentDictionaryBenchmark.cs index 4037a5c4753..cd757ec8465 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ConcurrentDictionaryBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ConcurrentDictionaryBenchmark.cs @@ -14,13 +14,13 @@ namespace Tsavorite.benchmark { - internal class KeyComparer : IEqualityComparer + internal class KeyComparer : IEqualityComparer { [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool Equals(Key x, Key y) => x.value == y.value; + public bool Equals(FixedLengthKey x, FixedLengthKey y) => x.value == y.value; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetHashCode(Key obj) => (int)Utility.GetHashCode(obj.value); + public int GetHashCode(FixedLengthKey obj) => (int)Utility.GetHashCode(obj.value); } internal unsafe class ConcurrentDictionary_YcsbBenchmark @@ -31,17 +31,17 @@ internal unsafe class ConcurrentDictionary_YcsbBenchmark readonly int readPercent, upsertPercent, rmwPercent; readonly Input[] input_; - readonly Key[] init_keys_; - readonly Key[] txn_keys_; + readonly FixedLengthKey[] init_keys_; + readonly FixedLengthKey[] txn_keys_; - readonly ConcurrentDictionary store; + readonly ConcurrentDictionary store; long idx_ = 0; long total_ops_done = 0; volatile bool done = false; Input* input_ptr; - internal ConcurrentDictionary_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLoader) + internal ConcurrentDictionary_YcsbBenchmark(FixedLengthKey[] i_keys_, FixedLengthKey[] t_keys_, TestLoader testLoader) { this.testLoader = testLoader; init_keys_ = i_keys_; @@ -90,7 +90,7 @@ private void RunYcsb(int thread_idx) } var sw = Stopwatch.StartNew(); - Value value = default; + FixedLengthValue value = default; long reads_done = 0; long writes_done = 0; long deletes_done = 0; @@ -129,7 +129,7 @@ private void RunYcsb(int thread_idx) } if (r < rmwPercent) { - store.AddOrUpdate(txn_keys_[idx], *(Value*)(input_ptr + (idx & 0x7)), (k, v) => new Value { value = v.value + (input_ptr + (idx & 0x7))->value }); + store.AddOrUpdate(txn_keys_[idx], *(FixedLengthValue*)(input_ptr + (idx & 0x7)), (k, v) => new FixedLengthValue { value = v.value + (input_ptr + (idx & 0x7))->value }); ++writes_done; continue; } @@ -267,7 +267,7 @@ private void SetupYcsb(int thread_idx) int count = 0; #endif - Value value = default; + FixedLengthValue value = default; for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; chunk_idx < testLoader.InitCount; @@ -276,7 +276,7 @@ private void SetupYcsb(int thread_idx) for (long idx = chunk_idx; idx < chunk_idx + YcsbConstants.kChunkSize; ++idx) { - Key key = init_keys_[idx]; + FixedLengthKey key = init_keys_[idx]; store[key] = value; } #if DASHBOARD @@ -371,14 +371,14 @@ void DoContinuousMeasurements() #region Load Data - internal static void CreateKeyVectors(TestLoader testLoader, out Key[] i_keys, out Key[] t_keys) + internal static void CreateKeyVectors(TestLoader testLoader, out FixedLengthKey[] i_keys, out FixedLengthKey[] t_keys) { - i_keys = new Key[testLoader.InitCount]; - t_keys = new Key[testLoader.TxnCount]; + i_keys = new FixedLengthKey[testLoader.InitCount]; + t_keys = new FixedLengthKey[testLoader.TxnCount]; } - internal class KeySetter : IKeySetter + internal class KeySetter : IKeySetter { - public void Set(Key[] vector, long idx, long value) => vector[idx].value = value; + public void Set(FixedLengthKey[] vector, long idx, long value) => vector[idx].value = value; } #endregion diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs index c74ad0335f9..ce7b5a413b3 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLenYcsbBenchmark.cs @@ -12,10 +12,22 @@ namespace Tsavorite.benchmark { #pragma warning disable IDE0065 // Misplaced using directive - using StructStoreFunctions = StoreFunctions>; + using StructStoreFunctions = StoreFunctions; - internal class Tsavorite_YcsbBenchmark + internal class FixedLenYcsbBenchmark { + RevivificationSettings FixedLengthBins = new() + { + FreeRecordBins = + [ + new RevivificationBin() + { + RecordSize = RecordInfo.GetLength() + 2 * (sizeof(int) + sizeof(long)), // We have "fixed length" for these integer bins, with long Key and Value + BestFitScanLimit = RevivificationBin.UseFirstFit + } + ] + }; + // Ensure sizes are aligned to chunk sizes static long InitCount; static long TxnCount; @@ -24,20 +36,20 @@ internal class Tsavorite_YcsbBenchmark readonly ManualResetEventSlim waiter = new(); readonly int numaStyle; readonly int readPercent, upsertPercent, rmwPercent; - readonly SessionFunctions functions; + readonly SessionFixedLenFunctions functions; readonly Input[] input_; - readonly Key[] init_keys_; - readonly Key[] txn_keys_; + readonly FixedLengthKey[] init_keys_; + readonly FixedLengthKey[] txn_keys_; readonly IDevice device; - readonly TsavoriteKV> store; + readonly TsavoriteKV> store; long idx_ = 0; long total_ops_done = 0; volatile bool done = false; - internal Tsavorite_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLoader) + internal FixedLenYcsbBenchmark(FixedLengthKey[] i_keys_, FixedLengthKey[] t_keys_, TestLoader testLoader) { if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) { @@ -54,7 +66,7 @@ internal Tsavorite_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLo readPercent = testLoader.ReadPercent; upsertPercent = testLoader.UpsertPercent; rmwPercent = testLoader.RmwPercent; - functions = new SessionFunctions(); + functions = new SessionFixedLenFunctions(); input_ = new Input[8]; for (int i = 0; i < 8; i++) @@ -64,7 +76,7 @@ internal Tsavorite_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLo { RevivificationLevel.None => default, RevivificationLevel.Chain => new RevivificationSettings(), - RevivificationLevel.Full => RevivificationSettings.DefaultFixedLength.Clone(), + RevivificationLevel.Full => FixedLengthBins, _ => throw new ApplicationException("Invalid RevivificationLevel") }; @@ -81,7 +93,7 @@ internal Tsavorite_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLo if (testLoader.Options.ThreadCount >= 16) device.ThrottleLimit = testLoader.Options.ThreadCount * 12; - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = testLoader.GetHashTableSize(), LogDevice = device, @@ -99,7 +111,7 @@ internal Tsavorite_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLo } store = new(kvSettings - , StoreFunctions.Create(new Key.Comparer()) + , StoreFunctions.Create(new FixedLengthKey.Comparer(), SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -125,7 +137,10 @@ private void RunYcsbUnsafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Value value = default; + FixedLengthKey keyStruct = default; + FixedLengthValue valueStruct = default; + ReadOnlySpan key = keyStruct.AsReadOnlySpan(); + Span value = valueStruct.AsSpan(); Input input = default; Output output = default; @@ -133,7 +148,8 @@ private void RunYcsbUnsafeContext(int thread_idx) long writes_done = 0; long deletes_done = 0; - using var session = store.NewSession(functions); + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); var uContext = session.UnsafeContext; uContext.BeginUnsafe(); @@ -157,26 +173,29 @@ private void RunYcsbUnsafeContext(int thread_idx) _ = uContext.CompletePending(false); } + keyStruct = txn_keys_[idx]; // Copy locally for SpanByte backing int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 if (r < readPercent) { - _ = uContext.Read(ref txn_keys_[idx], ref input, ref output, Empty.Default); + _ = uContext.Read(key, ref input, ref output, Empty.Default); ++reads_done; continue; } if (r < upsertPercent) { - _ = uContext.Upsert(ref txn_keys_[idx], ref value, Empty.Default); + _ = uContext.Upsert(key, value, Empty.Default); ++writes_done; continue; } if (r < rmwPercent) { - _ = uContext.RMW(ref txn_keys_[idx], ref input_[idx & 0x7], Empty.Default); + _ = uContext.RMW(key, ref input_[idx & 0x7], Empty.Default); ++writes_done; continue; } - _ = uContext.Delete(ref txn_keys_[idx], Empty.Default); + _ = uContext.Delete(key, Empty.Default); + if (di) + uContext.Upsert(key, value, Empty.Default); ++deletes_done; } } @@ -209,7 +228,10 @@ private void RunYcsbSafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Value value = default; + FixedLengthKey keyStruct = default; + FixedLengthValue valueStruct = default; + ReadOnlySpan key = keyStruct.AsReadOnlySpan(); + Span value = valueStruct.AsSpan(); Input input = default; Output output = default; @@ -217,7 +239,8 @@ private void RunYcsbSafeContext(int thread_idx) long writes_done = 0; long deletes_done = 0; - using var session = store.NewSession(functions); + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); var bContext = session.BasicContext; while (!done) @@ -235,26 +258,29 @@ private void RunYcsbSafeContext(int thread_idx) if (idx % 512 == 0) _ = bContext.CompletePending(false); + keyStruct = txn_keys_[idx]; // Copy locally for SpanByte backing int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 if (r < readPercent) { - _ = bContext.Read(ref txn_keys_[idx], ref input, ref output, Empty.Default); + _ = bContext.Read(key, ref input, ref output, Empty.Default); ++reads_done; continue; } if (r < upsertPercent) { - _ = bContext.Upsert(ref txn_keys_[idx], ref value, Empty.Default); + _ = bContext.Upsert(key, value, Empty.Default); ++writes_done; continue; } if (r < rmwPercent) { - _ = bContext.RMW(ref txn_keys_[idx], ref input_[idx & 0x7], Empty.Default); + _ = bContext.RMW(key, ref input_[idx & 0x7], Empty.Default); ++writes_done; continue; } - _ = bContext.Delete(ref txn_keys_[idx], Empty.Default); + _ = bContext.Delete(key, Empty.Default); + if (di) + bContext.Upsert(key, value, Empty.Default); ++deletes_done; } } @@ -389,11 +415,14 @@ private void SetupYcsbUnsafeContext(int thread_idx) } waiter.Wait(); - var session = store.NewSession(functions); + var session = store.NewSession(functions); var uContext = session.UnsafeContext; uContext.BeginUnsafe(); - Value value = default; + FixedLengthKey keyStruct = default; + FixedLengthValue valueStruct = default; + ReadOnlySpan key = keyStruct.AsReadOnlySpan(); + Span value = valueStruct.AsSpan(); try { @@ -410,7 +439,8 @@ private void SetupYcsbUnsafeContext(int thread_idx) _ = uContext.CompletePending(false); } - _ = uContext.Upsert(ref init_keys_[idx], ref value, Empty.Default); + keyStruct = txn_keys_[idx]; // Copy locally for SpanByte backing + _ = uContext.Upsert(key, value, Empty.Default); } } _ = uContext.CompletePending(true); @@ -433,10 +463,13 @@ private void SetupYcsbSafeContext(int thread_idx) } waiter.Wait(); - using var session = store.NewSession(functions); + using var session = store.NewSession(functions); var bContext = session.BasicContext; - Value value = default; + FixedLengthKey keyStruct = default; + FixedLengthValue valueStruct = default; + ReadOnlySpan key = keyStruct.AsReadOnlySpan(); + Span value = valueStruct.AsSpan(); for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; chunk_idx < InitCount; @@ -451,7 +484,8 @@ private void SetupYcsbSafeContext(int thread_idx) _ = bContext.CompletePending(false); } - _ = bContext.Upsert(ref init_keys_[idx], ref value, Empty.Default); + keyStruct = txn_keys_[idx]; // Copy locally for SpanByte backing + _ = bContext.Upsert(key, value, Empty.Default); } } @@ -460,18 +494,18 @@ private void SetupYcsbSafeContext(int thread_idx) #region Load Data - internal static void CreateKeyVectors(TestLoader testLoader, out Key[] i_keys, out Key[] t_keys) + internal static void CreateKeyVectors(TestLoader testLoader, out FixedLengthKey[] i_keys, out FixedLengthKey[] t_keys) { InitCount = YcsbConstants.kChunkSize * (testLoader.InitCount / YcsbConstants.kChunkSize); TxnCount = YcsbConstants.kChunkSize * (testLoader.TxnCount / YcsbConstants.kChunkSize); - i_keys = new Key[InitCount]; - t_keys = new Key[TxnCount]; + i_keys = new FixedLengthKey[InitCount]; + t_keys = new FixedLengthKey[TxnCount]; } - internal class KeySetter : IKeySetter + internal class KeySetter : IKeySetter { - public void Set(Key[] vector, long idx, long value) => vector[idx].value = value; + public void Set(FixedLengthKey[] vector, long idx, long value) => vector[idx].value = value; } #endregion diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs new file mode 100644 index 00000000000..4c66d867020 --- /dev/null +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthKey.cs @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using Tsavorite.core; + +namespace Tsavorite.benchmark +{ + [StructLayout(LayoutKind.Explicit, Size = sizeof(long))] + public struct FixedLengthKey + { + [FieldOffset(0)] + public long value; + + public override 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)); + + public struct Comparer : IKeyComparer + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetHashCode64(ReadOnlySpan key) => Utility.GetHashCode(key.AsRef().value); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(ReadOnlySpan key1, ReadOnlySpankey2) => key1.AsRef().value == key2.AsRef().value; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Value.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthValue.cs similarity index 59% rename from libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Value.cs rename to libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthValue.cs index 0e93c207c7c..c4a170bc37b 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Value.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/FixedLengthValue.cs @@ -5,15 +5,20 @@ //#define FIXED_SIZE_VALUE //#define FIXED_SIZE_VALUE_WITH_LOCK +using System; +using System.Runtime.CompilerServices; using System.Runtime.InteropServices; namespace Tsavorite.benchmark { [StructLayout(LayoutKind.Explicit, Size = 8)] - public struct Value + public struct FixedLengthValue { public const int Size = 8; + // Only call this for stack-based structs, not the ones in the *_keys vectors + public unsafe Span AsSpan() => new(Unsafe.AsPointer(ref this), sizeof(long)); + [FieldOffset(0)] public long value; } diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Key.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Key.cs deleted file mode 100644 index 708026612c6..00000000000 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Key.cs +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; -using Tsavorite.core; - -namespace Tsavorite.benchmark -{ - [StructLayout(LayoutKind.Explicit, Size = 8)] - public struct Key - { - [FieldOffset(0)] - public long value; - - public override string ToString() => "{ " + value + " }"; - - public struct Comparer : IKeyComparer - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetHashCode64(ref Key key) => Utility.GetHashCode(key.value); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool Equals(ref Key key1, ref Key key2) => key1.value == key2.value; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs index 433a30fa57a..8a2bdc05328 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/KeySpanByte.cs @@ -1,16 +1,39 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; +using System.Runtime.CompilerServices; using System.Runtime.InteropServices; +using Tsavorite.core; namespace Tsavorite.benchmark { - [StructLayout(LayoutKind.Explicit, Size = SpanByteYcsbBenchmark.kKeySize)] + [StructLayout(LayoutKind.Explicit, Size = DataSize)] public struct KeySpanByte { + public const int DataSize = 12; + public const int TotalSize = DataSize + sizeof(int); + + /// The data of the key [FieldOffset(0)] - public int length; - [FieldOffset(4)] public long value; + + /// + /// 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. + /// + [FieldOffset(sizeof(long))] + public int padding; + + // Only call this for stack-based structs, not the ones in the *_keys vectors + public override string ToString() => "{ " + value + " }"; + + public unsafe ReadOnlySpan AsReadOnlySpan() => new(Unsafe.AsPointer(ref this), DataSize); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs new file mode 100644 index 00000000000..5eca13c6a59 --- /dev/null +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectValue.cs @@ -0,0 +1,30 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +#define EIGHT_BYTE_VALUE +//#define FIXED_SIZE_VALUE +//#define FIXED_SIZE_VALUE_WITH_LOCK + +using Tsavorite.core; + +namespace Tsavorite.benchmark +{ + public class ObjectValue : IHeapObject + { + 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 void Dispose() { } + + public override string ToString() => value.ToString(); + + public class Serializer : BinaryObjectSerializer + { + public override void Deserialize(out IHeapObject obj) => obj = new ObjectValue { value = reader.ReadInt32() }; + + public override void Serialize(IHeapObject obj) => writer.Write(((ObjectValue)obj).value); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs new file mode 100644 index 00000000000..2ca66308768 --- /dev/null +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/ObjectYcsbBenchmark.cs @@ -0,0 +1,530 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.InteropServices; +using System.Threading; +using Tsavorite.core; + +#pragma warning disable IDE0007 // Use implicit type + +namespace Tsavorite.benchmark +{ +#pragma warning disable IDE0065 // Misplaced using directive + using ObjectStoreFunctions = StoreFunctions; + + internal class ObjectYcsbBenchmark + { + // Ensure sizes are aligned to chunk sizes + static long InitCount; + static long TxnCount; + + readonly TestLoader testLoader; + readonly ManualResetEventSlim waiter = new(); + readonly int numaStyle; + readonly int readPercent, upsertPercent, rmwPercent; + readonly SessionObjectFunctions functions; + readonly Input[] input_; + + readonly FixedLengthKey[] init_keys_; + readonly FixedLengthKey[] txn_keys_; + readonly ObjectValue[] object_values; // In parallel with init_keys_ + + readonly IDevice device; + readonly TsavoriteKV> store; + + long idx_ = 0; + long total_ops_done = 0; + volatile bool done = false; + + internal const int kValueDataSize = SpanByteYcsbBenchmark.kValueDataSize; + + internal ObjectYcsbBenchmark(FixedLengthKey[] i_keys_, FixedLengthKey[] t_keys_, TestLoader testLoader) + { + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + // Affinize main thread to last core on first socket if not used by experiment + var (numGrps, numProcs) = Native32.GetNumGroupsProcsPerGroup(); + if ((testLoader.Options.NumaStyle == 0 && testLoader.Options.ThreadCount <= (numProcs - 1)) || + (testLoader.Options.NumaStyle == 1 && testLoader.Options.ThreadCount <= numGrps * (numProcs - 1))) + Native32.AffinitizeThreadRoundRobin(numProcs - 1); + } + this.testLoader = testLoader; + init_keys_ = i_keys_; + txn_keys_ = t_keys_; + if (testLoader.Options.UseObjectValues) + object_values = new ObjectValue[InitCount]; + numaStyle = testLoader.Options.NumaStyle; + readPercent = testLoader.ReadPercent; + upsertPercent = testLoader.UpsertPercent; + rmwPercent = testLoader.RmwPercent; + functions = new SessionObjectFunctions(); + + input_ = new Input[8]; + for (int i = 0; i < 8; i++) + input_[i].value = i; + + var revivificationSettings = testLoader.Options.RevivificationLevel switch + { + RevivificationLevel.None => default, + RevivificationLevel.Chain => new RevivificationSettings(), + RevivificationLevel.Full => new RevivificationSettings() + { + FreeRecordBins = + [ + new RevivificationBin() + { + RecordSize = RecordInfo.GetLength() + KeySpanByte.TotalSize + kValueDataSize + 8, // extra to ensure rounding up of value + NumberOfRecords = testLoader.Options.RevivBinRecordCount, + BestFitScanLimit = RevivificationBin.UseFirstFit + } + ], + }, + _ => throw new ApplicationException("Invalid RevivificationLevel") + }; + + if (revivificationSettings is not null) + { + revivificationSettings.RevivifiableFraction = testLoader.Options.RevivifiableFraction; + revivificationSettings.RestoreDeletedRecordsIfBinIsFull = true; + } + + device = Devices.CreateLogDevice(TestLoader.DevicePath, preallocateFile: true, deleteOnClose: !testLoader.RecoverMode, useIoCompletionPort: true); + + var kvSettings = new KVSettings() + { + IndexSize = testLoader.GetHashTableSize(), + LogDevice = device, + PreallocateLog = true, + MemorySize = 1L << 35, + RevivificationSettings = revivificationSettings, + CheckpointDir = testLoader.BackupPath, + MaxInlineValueSize = testLoader.Options.UseOverflowValues ? 64 : 128 + }; + + if (testLoader.Options.UseSmallMemoryLog) + { + kvSettings.PageSize = 1L << 22; + kvSettings.SegmentSize = 1L << 26; + kvSettings.MemorySize = 1L << 26; + } + + store = new(kvSettings + , StoreFunctions.Create(SpanByteComparer.Instance, DefaultRecordDisposer.Instance) + , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) + ); + } + + internal void Dispose() + { + store.Dispose(); + device.Dispose(); + } + + private void RunYcsbUnsafeContext(int thread_idx) + { + RandomGenerator rng = new((uint)(1 + thread_idx)); + + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + if (numaStyle == 0) + Native32.AffinitizeThreadRoundRobin((uint)thread_idx); + else + Native32.AffinitizeThreadShardedNuma((uint)thread_idx, 2); // assuming two NUMA sockets + } + waiter.Wait(); + + var sw = Stopwatch.StartNew(); + + Span value = stackalloc byte[kValueDataSize]; + Span input = stackalloc byte[kValueDataSize]; + Span output = stackalloc byte[kValueDataSize]; + + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); + + long reads_done = 0; + long writes_done = 0; + long deletes_done = 0; + + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); + var uContext = session.UnsafeContext; + uContext.BeginUnsafe(); + + try + { + while (!done) + { + long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + while (chunk_idx >= TxnCount) + { + if (chunk_idx == TxnCount) + idx_ = 0; + chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + } + + for (long idx = chunk_idx; idx < chunk_idx + YcsbConstants.kChunkSize && !done; ++idx) + { + if (idx % 512 == 0) + { + uContext.Refresh(); + uContext.CompletePending(false); + } + + unsafe + { + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = txn_keys_[idx].AsReadOnlySpan(); + + int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 + if (r < readPercent) + { + uContext.Read(key, ref pinnedInputSpan, ref _output, Empty.Default); + ++reads_done; + continue; + } + if (r < upsertPercent) + { + uContext.Upsert(key, value, Empty.Default); + ++writes_done; + continue; + } + if (r < rmwPercent) + { + uContext.RMW(key, ref pinnedInputSpan, Empty.Default); + ++writes_done; + continue; + } + uContext.Delete(key, Empty.Default); + if (di) + uContext.Upsert(key, value, Empty.Default); + ++deletes_done; + } + } + } + + uContext.CompletePending(true); + } + finally + { + uContext.EndUnsafe(); + } + + sw.Stop(); + + Console.WriteLine($"Thread {thread_idx} done; {reads_done} reads, {writes_done} writes, {deletes_done} deletes in {sw.ElapsedMilliseconds} ms."); + Interlocked.Add(ref total_ops_done, reads_done + writes_done + deletes_done); + } + + private void RunYcsbSafeContext(int thread_idx) + { + RandomGenerator rng = new((uint)(1 + thread_idx)); + + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + if (numaStyle == 0) + Native32.AffinitizeThreadRoundRobin((uint)thread_idx); + else + Native32.AffinitizeThreadShardedNuma((uint)thread_idx, 2); // assuming two NUMA sockets + } + waiter.Wait(); + + var sw = Stopwatch.StartNew(); + + Span value = stackalloc byte[kValueDataSize]; + Span input = stackalloc byte[kValueDataSize]; + Span output = stackalloc byte[kValueDataSize]; + + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); + + long reads_done = 0; + long writes_done = 0; + long deletes_done = 0; + + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); + var bContext = session.BasicContext; + + while (!done) + { + long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + while (chunk_idx >= TxnCount) + { + if (chunk_idx == TxnCount) + idx_ = 0; + chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + } + + for (long idx = chunk_idx; idx < chunk_idx + YcsbConstants.kChunkSize && !done; ++idx) + { + if (idx % 512 == 0) + { + if (!testLoader.Options.UseSafeContext) + bContext.Refresh(); + bContext.CompletePending(false); + } + + unsafe + { + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = txn_keys_[idx].AsReadOnlySpan(); + + int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 + if (r < readPercent) + { + bContext.Read(key, ref pinnedInputSpan, ref _output, Empty.Default); + ++reads_done; + continue; + } + if (r < upsertPercent) + { + bContext.Upsert(key, value, Empty.Default); + ++writes_done; + continue; + } + if (r < rmwPercent) + { + bContext.RMW(key, ref pinnedInputSpan, Empty.Default); + ++writes_done; + continue; + } + bContext.Delete(key, Empty.Default); + if (di) + bContext.Upsert(key, value, Empty.Default); + ++deletes_done; + } + } + } + + bContext.CompletePending(true); + + sw.Stop(); + + Console.WriteLine($"Thread {thread_idx} done; {reads_done} reads, {writes_done} writes, {deletes_done} deletes in {sw.ElapsedMilliseconds} ms."); + Interlocked.Add(ref total_ops_done, reads_done + writes_done + deletes_done); + } + + internal unsafe (double insPerSec, double opsPerSec, long tailAddress) Run(TestLoader testLoader) + { + Thread[] workers = new Thread[testLoader.Options.ThreadCount]; + + Console.WriteLine("Executing setup."); + + var storeWasRecovered = testLoader.MaybeRecoverStore(store); + long elapsedMs = 0; + if (!storeWasRecovered) + { + // Setup the store for the YCSB benchmark. + Console.WriteLine("Loading TsavoriteKV from data"); + for (int idx = 0; idx < testLoader.Options.ThreadCount; ++idx) + { + int x = idx; + if (testLoader.Options.UseSafeContext) + workers[idx] = new Thread(() => SetupYcsbSafeContext(x)); + else + workers[idx] = new Thread(() => SetupYcsbUnsafeContext(x)); + } + + foreach (Thread worker in workers) + worker.Start(); + + waiter.Set(); + var sw = Stopwatch.StartNew(); + foreach (Thread worker in workers) + worker.Join(); + + sw.Stop(); + waiter.Reset(); + + elapsedMs = sw.ElapsedMilliseconds; + } + double insertsPerSecond = elapsedMs == 0 ? 0 : ((double)InitCount / elapsedMs) * 1000; + Console.WriteLine(TestStats.GetLoadingTimeLine(insertsPerSecond, elapsedMs)); + Console.WriteLine(TestStats.GetAddressesLine(AddressLineNum.Before, store.Log.BeginAddress, store.Log.HeadAddress, store.Log.ReadOnlyAddress, store.Log.TailAddress)); + + if (!storeWasRecovered) + testLoader.MaybeCheckpointStore(store); + + // Uncomment below to dispose log from memory, use for 100% read workloads only + // store.Log.DisposeFromMemory(); + + idx_ = 0; + + if (testLoader.Options.DumpDistribution) + Console.WriteLine(store.DumpDistribution()); + + // Ensure first fold-over checkpoint is fast + if (testLoader.Options.PeriodicCheckpointMilliseconds > 0 && testLoader.Options.PeriodicCheckpointType == CheckpointType.FoldOver) + store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, true); + + Console.WriteLine("Executing experiment."); + + // Run the experiment. + for (int idx = 0; idx < testLoader.Options.ThreadCount; ++idx) + { + int x = idx; + if (testLoader.Options.UseSafeContext) + workers[idx] = new Thread(() => RunYcsbSafeContext(x)); + else + workers[idx] = new Thread(() => RunYcsbUnsafeContext(x)); + } + + // Start threads. + foreach (Thread worker in workers) + worker.Start(); + + waiter.Set(); + var swatch = Stopwatch.StartNew(); + + if (testLoader.Options.PeriodicCheckpointMilliseconds <= 0) + { + Thread.Sleep(TimeSpan.FromSeconds(testLoader.Options.RunSeconds)); + } + else + { + var checkpointTaken = 0; + while (swatch.ElapsedMilliseconds < 1000 * testLoader.Options.RunSeconds) + { + if (checkpointTaken < swatch.ElapsedMilliseconds / testLoader.Options.PeriodicCheckpointMilliseconds) + { + long start = swatch.ElapsedTicks; + if (store.TryInitiateHybridLogCheckpoint(out _, testLoader.Options.PeriodicCheckpointType, testLoader.Options.PeriodicCheckpointTryIncremental)) + { + store.CompleteCheckpointAsync().AsTask().GetAwaiter().GetResult(); + var timeTaken = (swatch.ElapsedTicks - start) / TimeSpan.TicksPerMillisecond; + Console.WriteLine("Checkpoint time: {0}ms", timeTaken); + checkpointTaken++; + } + } + } + Console.WriteLine($"Checkpoint taken {checkpointTaken}"); + } + + swatch.Stop(); + + done = true; + foreach (Thread worker in workers) + worker.Join(); + + waiter.Reset(); + + double seconds = swatch.ElapsedMilliseconds / 1000.0; + Console.WriteLine(TestStats.GetAddressesLine(AddressLineNum.After, store.Log.BeginAddress, store.Log.HeadAddress, store.Log.ReadOnlyAddress, store.Log.TailAddress)); + + double opsPerSecond = total_ops_done / seconds; + Console.WriteLine(TestStats.GetTotalOpsString(total_ops_done, seconds)); + Console.WriteLine(TestStats.GetStatsLine(StatsLineNum.Iteration, YcsbConstants.OpsPerSec, opsPerSecond)); + return (insertsPerSecond, opsPerSecond, store.Log.TailAddress); + } + + private void SetupYcsbUnsafeContext(int thread_idx) + { + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + if (numaStyle == 0) + Native32.AffinitizeThreadRoundRobin((uint)thread_idx); + else + Native32.AffinitizeThreadShardedNuma((uint)thread_idx, 2); // assuming two NUMA sockets + } + waiter.Wait(); + + using var session = store.NewSession(functions); + var uContext = session.UnsafeContext; + uContext.BeginUnsafe(); + + Span value = stackalloc byte[kValueDataSize]; + + try + { + for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + chunk_idx < InitCount; + chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize) + { + for (long idx = chunk_idx; idx < chunk_idx + YcsbConstants.kChunkSize; ++idx) + { + if (idx % 256 == 0) + { + uContext.Refresh(); + if (idx % 65536 == 0) + uContext.CompletePending(false); + } + + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = init_keys_[idx].AsReadOnlySpan(); + if (object_values is null) + uContext.Upsert(key, value, Empty.Default); + else + uContext.Upsert(key, object_values[idx] = new ObjectValue() { value = init_keys_[idx].value }, Empty.Default); + } + } + uContext.CompletePending(true); + } + finally + { + uContext.EndUnsafe(); + } + } + + private void SetupYcsbSafeContext(int thread_idx) + { + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + if (numaStyle == 0) + Native32.AffinitizeThreadRoundRobin((uint)thread_idx); + else + Native32.AffinitizeThreadShardedNuma((uint)thread_idx, 2); // assuming two NUMA sockets + } + waiter.Wait(); + + using var session = store.NewSession(functions); + var bContext = session.BasicContext; + + Span value = stackalloc byte[kValueDataSize]; + + for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; + chunk_idx < InitCount; + chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize) + { + for (long idx = chunk_idx; idx < chunk_idx + YcsbConstants.kChunkSize; ++idx) + { + if (idx % 256 == 0) + { + bContext.Refresh(); + if (idx % 65536 == 0) + bContext.CompletePending(false); + } + + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = init_keys_[idx].AsReadOnlySpan(); + if (object_values is null) + bContext.Upsert(key, value, Empty.Default); + else + bContext.Upsert(key, object_values[idx] = new ObjectValue() { value = init_keys_[idx].value }, Empty.Default); + } + } + + 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); + TxnCount = YcsbConstants.kChunkSize * (testLoader.TxnCount / YcsbConstants.kChunkSize); + + 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 + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs index 375e2b6a549..74252ab45c9 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Options.cs @@ -11,9 +11,10 @@ class Options { [Option('b', "benchmark", Required = false, Default = 0, HelpText = "Benchmark to run:" + - "\n 0 = YCSB" + - "\n 1 = YCSB with SpanByte" + - "\n 2 = ConcurrentDictionary")] + "\n 0 = YCSB with Fixed-length (long- and int-sized) SpanByte values" + + "\n 1 = YCSB with longer SpanByte keys and values" + + "\n 2 = YCSB with longer values that may also be represented as overflow byte[] or as Object" + + "\n 3 = ConcurrentDictionary")] public int Benchmark { get; set; } [Option('t', "threads", Required = false, Default = 8, @@ -62,6 +63,10 @@ class Options " # (one value): All bins have this number of records, else error")] public int RevivBinRecordCount { get; set; } + [Option("di", Required = false, Default = false, + HelpText = "Delete+insert; immediately reinsert the key after deleting it")] + public bool DeleteAndReinsert { get; set; } + [Option("reviv-mutable%", Separator = ',', Required = false, Default = RevivificationSettings.DefaultRevivifiableFraction, HelpText = "Percentage of in-memory region that is eligible for revivification")] public double RevivifiableFraction { get; set; } @@ -82,6 +87,14 @@ class Options HelpText = "Use Small Memory log in experiment")] public bool UseSmallMemoryLog { get; set; } + [Option("ov", Required = false, Default = false, + HelpText = "Use Small MaxInlineValueSize in SpanByte benchmark to test (o)verflow (v)alue allocations")] + public bool UseOverflowValues { get; set; } + + [Option("obj", Required = false, Default = false, + HelpText = "Use (obj)ect values")] + public bool UseObjectValues { get; set; } + [Option("hashpack", Required = false, Default = 2.0, HelpText = "The hash table packing; divide the number of keys by this to cause hash collisions")] public double HashPacking { get; set; } @@ -91,7 +104,7 @@ class Options public bool UseSafeContext { get; set; } [Option("chkptms", Required = false, Default = 0, - HelpText = "If > 0, the number of milliseconds between checkpoints in experiment (else checkpointing is not done")] + HelpText = "If > 0, the number of milliseconds between checkpoints in experiment (else checkpointing is not done)")] public int PeriodicCheckpointMilliseconds { get; set; } [Option("chkptsnap", Required = false, Default = false, @@ -112,7 +125,7 @@ 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};" - + $" revivfrac {RevivifiableFraction}; t: {ThreadCount}; i: {IterationCount}; hp: {HashPacking};" + + $" 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)};" + $" chkptms: {PeriodicCheckpointMilliseconds}; chkpttype: {(PeriodicCheckpointMilliseconds > 0 ? PeriodicCheckpointType.ToString() : "None")};" + $" chkptincr: {boolStr(PeriodicCheckpointTryIncremental)}"; diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Output.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Output.cs index 45c562f6949..b2beeb8b577 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Output.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Output.cs @@ -11,6 +11,6 @@ namespace Tsavorite.benchmark public struct Output { [FieldOffset(0)] - public Value value; + public FixedLengthValue value; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs index ce329fcdb7b..7c247f334d3 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/Program.cs @@ -28,9 +28,9 @@ public static void Main(string[] args) switch (testLoader.BenchmarkType) { - case BenchmarkType.Ycsb: + case BenchmarkType.FixedLen: { - var tester = new Tsavorite_YcsbBenchmark(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(); } @@ -42,7 +42,14 @@ public static void Main(string[] args) tester.Dispose(); } break; - case BenchmarkType.ConcurrentDictionaryYcsb: + case BenchmarkType.Object: + { + var tester = new ObjectYcsbBenchmark(testLoader.init_keys, testLoader.txn_keys, testLoader); + testStats.AddResult(tester.Run(testLoader)); + tester.Dispose(); + } + break; + case BenchmarkType.ConcurrentDictionary: { var tester = new ConcurrentDictionary_YcsbBenchmark(testLoader.init_keys, testLoader.txn_keys, testLoader); testStats.AddResult(tester.Run(testLoader)); diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs new file mode 100644 index 00000000000..2c295df7f2b --- /dev/null +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFixedLenFunctions.cs @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using Tsavorite.core; + +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 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(ref 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 bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; + + // Upsert functions + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InitialWriter(ref LogRecord logRecord, ref 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, ref RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) + { + logRecord.TrySetValueObject(srcValue, ref sizeInfo); + return true; + } + + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref Input input, ref TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + => true; // not used + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InPlaceWriter(ref LogRecord logRecord, ref 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, ref RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) + { + logRecord.TrySetValueObject(srcValue, ref sizeInfo); + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InPlaceWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref Input input, ref TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + return dstLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + } + + // RMW functions + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InitialUpdater(ref LogRecord logRecord, ref 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, ref 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(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref 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(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref 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 void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref Input input, ref Output output, ref RMWInfo rmwInfo) { } + + public bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref Input input, ref Output output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public RecordFieldInfo GetRMWModifiedFieldInfo(ref 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(); + + /// Length of value object, when populated by Upsert using given value and input + public 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 }; + + /// Length of value object, when populated by Upsert using given log record and input + public unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref Input input) + where TSourceLogRecord : ISourceLogRecord + => throw new NotImplementedException("GetUpsertFieldInfo(TSourceLogRecord)"); + + static unsafe RecordFieldInfo GetFieldInfo() => new () { KeyDataSize = sizeof(FixedLengthKey), ValueDataSize = sizeof(FixedLengthValue) }; + + public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { } + + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref Input input, ReadOnlySpan srcValue, ref Output output, ref UpsertInfo upsertInfo) { } + + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref Input input, IHeapObject srcValue, ref Output output, ref UpsertInfo upsertInfo) { } + + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref Input input, ref TSourceLogRecord inputLogRecord, ref Output output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { } + + public void ConvertOutputToHeap(ref Input input, ref Output output) { } + } + + static class StaticUtilities + { + public static unsafe ref T AsRef(this Span spanByte) where T : unmanaged + { + Debug.Assert(spanByte.Length == Unsafe.SizeOf()); + return ref Unsafe.As(ref spanByte[0]); + } + + public static ref readonly T AsRef(this ReadOnlySpan spanByte) where T : unmanaged + { + Debug.Assert(spanByte.Length == Unsafe.SizeOf()); + return ref MemoryMarshal.Cast(spanByte)[0]; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFunctions.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFunctions.cs deleted file mode 100644 index db536505bfe..00000000000 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionFunctions.cs +++ /dev/null @@ -1,93 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Runtime.CompilerServices; -using Tsavorite.core; - -namespace Tsavorite.benchmark -{ - public struct SessionFunctions : ISessionFunctions - { - public void RMWCompletionCallback(ref Key key, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) - { - } - - public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, Empty ctx, Status status, RecordMetadata recordMetadata) - { - } - - // Read functions - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool SingleReader(ref Key key, ref Input input, ref Value value, ref Output dst, ref ReadInfo readInfo) - { - dst.value = value; - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool ConcurrentReader(ref Key key, ref Input input, ref Value value, ref Output dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - dst.value = value; - return true; - } - - public bool SingleDeleter(ref Key key, ref Value value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) { value = default; return true; } - - public bool ConcurrentDeleter(ref Key key, ref Value value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - - // Upsert functions - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool SingleWriter(ref Key key, ref Input input, ref Value src, ref Value dst, ref Output output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool ConcurrentWriter(ref Key key, ref Input input, ref Value src, ref Value dst, ref Output output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - dst = src; - return true; - } - - // RMW functions - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InitialUpdater(ref Key key, ref Input input, ref Value value, ref Output output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value.value = input.value; - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value, ref Output output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value.value += input.value; - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue, ref Output output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue.value = input.value + oldValue.value; - return true; - } - - public bool PostCopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue, ref Output output, ref RMWInfo rmwInfo) => true; - - public bool NeedInitialUpdate(ref Key key, ref Input input, ref Output output, ref RMWInfo rmwInfo) => true; - - public void PostInitialUpdater(ref Key key, ref Input input, ref Value value, ref Output output, ref RMWInfo rmwInfo) { } - - public bool NeedCopyUpdate(ref Key key, ref Input input, ref Value oldValue, ref Output output, ref RMWInfo rmwInfo) => true; - - public int GetRMWModifiedValueLength(ref Value value, ref Input input) => 0; - public int GetRMWInitialValueLength(ref Input input) => 0; - public int GetUpsertValueLength(ref Value value, ref Input input) => Value.Size; - - public void PostSingleDeleter(ref Key key, ref DeleteInfo deleteInfo) { } - - public void PostSingleWriter(ref Key key, ref Input input, ref Value src, ref Value dst, ref Output output, ref UpsertInfo upsertInfo, WriteReason reason) { } - - public void ConvertOutputToHeap(ref Input input, ref Output output) { } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionObjectFunctions.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionObjectFunctions.cs new file mode 100644 index 00000000000..6b27c673872 --- /dev/null +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionObjectFunctions.cs @@ -0,0 +1,72 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; + +namespace Tsavorite.benchmark +{ + public sealed class SessionObjectFunctions : SpanByteFunctions + { + /// + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref ReadInfo readInfo) + { + if (!srcLogRecord.Info.ValueIsObject) + srcLogRecord.ValueSpan.CopyTo(output.SpanByte.Span); + else // Slice the output because it is a larger buffer + output.SpanByte.Span.Slice(0, sizeof(long)).AsRef() = ((ObjectValue)srcLogRecord.ValueObject).value; + return true; + } + + // Note: Currently, only the ReadOnlySpan form of InPlaceWriter value is used here. + + /// + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration + if (!logRecord.Info.ValueIsObject) // If !ValueIsObject, the destination data length, either inline or out-of-line, should already be sufficient + srcValue.CopyTo(logRecord.ValueSpan); + else // Slice the input because it comes from a larger buffer + ((ObjectValue)logRecord.ValueObject).value = srcValue.Slice(0, FixedLengthValue.Size).AsRef().value; + return true; + } + + /// + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration + if (dstLogRecord.Info.ValueIsInline && srcValue.Length <= dstLogRecord.ValueSpan.Length) + srcValue.CopyTo(dstLogRecord.ValueSpan); + else if (!dstLogRecord.Info.ValueIsObject) // process overflow + return dstLogRecord.TrySetValueSpan(srcValue, ref sizeInfo); + else // Slice the input because it comes from a larger buffer + ((ObjectValue)dstLogRecord.ValueObject).value = srcValue.Slice(0, FixedLengthValue.Size).AsRef().value; + return true; + } + + /// + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, IHeapObject srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration. It is called only during Setup. + return dstLogRecord.TrySetValueObject(srcValue, ref sizeInfo); + } + + /// + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) => throw new TsavoriteException("InitialUpdater not implemented for YCSB"); + + /// + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + => InPlaceUpdater(ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); + + /// + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + { + // This does not try to set ETag or Expiration + if (!logRecord.Info.ValueIsObject) // If !ValueIsObject, the destination data length, either inline or out-of-line, should already be sufficient + input.CopyTo(logRecord.ValueSpan); + else // Slice the input because it comes from a larger buffer + ((ObjectValue)logRecord.ValueObject).value = input.ReadOnlySpan.Slice(0, FixedLengthValue.Size).AsRef().value; + return true; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionSpanByteFunctions.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionSpanByteFunctions.cs index 8accfc7883f..59c20d1b58b 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionSpanByteFunctions.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SessionSpanByteFunctions.cs @@ -1,11 +1,55 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using Tsavorite.core; namespace Tsavorite.benchmark { public sealed class SessionSpanByteFunctions : SpanByteFunctions { + /// + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref ReadInfo readInfo) + { + srcLogRecord.ValueSpan.CopyTo(output.SpanByte.Span); + return true; + } + + // Note: Currently, only the ReadOnlySpan form of Upsert value is used here. + + /// + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration + srcValue.CopyTo(logRecord.ValueSpan); + return true; + } + + /// + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration + srcValue.CopyTo(dstLogRecord.ValueSpan); + return true; + } + + /// + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) => throw new TsavoriteException("InitialUpdater not implemented for YCSB"); + + /// + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + { + // This does not try to set ETag or Expiration + input.CopyTo(dstLogRecord.ValueSpan); + return true; + } + + /// + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + { + // This does not try to set ETag or Expiration + input.CopyTo(logRecord.ValueSpan); + return true; + } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs index e19235e10c7..9a6e2934dd4 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/SpanByteYcsbBenchmark.cs @@ -12,7 +12,7 @@ namespace Tsavorite.benchmark { #pragma warning disable IDE0065 // Misplaced using directive - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; internal class SpanByteYcsbBenchmark { @@ -31,14 +31,13 @@ internal class SpanByteYcsbBenchmark readonly KeySpanByte[] txn_keys_; readonly IDevice device; - readonly TsavoriteKV> store; + readonly TsavoriteKV> store; long idx_ = 0; long total_ops_done = 0; volatile bool done = false; - internal const int kKeySize = 16; - internal const int kValueSize = 100; + internal const int kValueDataSize = 96; // 100 minus 4-byte length prefix. internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, TestLoader testLoader) { @@ -73,7 +72,7 @@ internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, Tes [ new RevivificationBin() { - RecordSize = RecordInfo.GetLength() + kKeySize + kValueSize + 8, // extra to ensure rounding up of value + RecordSize = RecordInfo.GetLength() + KeySpanByte.TotalSize + kValueDataSize + 8, // extra to ensure rounding up of value NumberOfRecords = testLoader.Options.RevivBinRecordCount, BestFitScanLimit = RevivificationBin.UseFirstFit } @@ -90,7 +89,7 @@ internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, Tes device = Devices.CreateLogDevice(TestLoader.DevicePath, preallocateFile: true, deleteOnClose: !testLoader.RecoverMode, useIoCompletionPort: true); - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = testLoader.GetHashTableSize(), LogDevice = device, @@ -108,7 +107,7 @@ internal SpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys_, Tes } store = new(kvSettings - , StoreFunctions.Create() + , StoreFunctions.Create() , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -134,19 +133,19 @@ private void RunYcsbUnsafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Span value = stackalloc byte[kValueSize]; - Span input = stackalloc byte[kValueSize]; - Span output = stackalloc byte[kValueSize]; + Span value = stackalloc byte[kValueDataSize]; + Span input = stackalloc byte[kValueDataSize]; + Span output = stackalloc byte[kValueDataSize]; - SpanByte _value = SpanByte.FromPinnedSpan(value); - SpanByte _input = SpanByte.FromPinnedSpan(input); + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); long reads_done = 0; long writes_done = 0; long deletes_done = 0; - using var session = store.NewSession(functions); + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); var uContext = session.UnsafeContext; uContext.BeginUnsafe(); @@ -170,27 +169,35 @@ private void RunYcsbUnsafeContext(int thread_idx) uContext.CompletePending(false); } - int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 - if (r < readPercent) - { - uContext.Read(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _input, ref _output, Empty.Default); - ++reads_done; - continue; - } - if (r < upsertPercent) + unsafe { - uContext.Upsert(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _value, Empty.Default); - ++writes_done; - continue; - } - if (r < rmwPercent) - { - uContext.RMW(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _input, Empty.Default); - ++writes_done; - continue; + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = txn_keys_[idx].AsReadOnlySpan(); + + int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 + if (r < readPercent) + { + uContext.Read(key, ref pinnedInputSpan, ref _output, Empty.Default); + ++reads_done; + continue; + } + if (r < upsertPercent) + { + uContext.Upsert(key, value, Empty.Default); + ++writes_done; + continue; + } + if (r < rmwPercent) + { + uContext.RMW(key, ref pinnedInputSpan, Empty.Default); + ++writes_done; + continue; + } + uContext.Delete(key, Empty.Default); + if (di) + uContext.Upsert(key, value, Empty.Default); + ++deletes_done; } - uContext.Delete(ref SpanByte.Reinterpret(ref txn_keys_[idx]), Empty.Default); - ++deletes_done; } } @@ -222,19 +229,19 @@ private void RunYcsbSafeContext(int thread_idx) var sw = Stopwatch.StartNew(); - Span value = stackalloc byte[kValueSize]; - Span input = stackalloc byte[kValueSize]; - Span output = stackalloc byte[kValueSize]; + Span value = stackalloc byte[kValueDataSize]; + Span input = stackalloc byte[kValueDataSize]; + Span output = stackalloc byte[kValueDataSize]; - SpanByte _value = SpanByte.FromPinnedSpan(value); - SpanByte _input = SpanByte.FromPinnedSpan(input); + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory _output = SpanByteAndMemory.FromPinnedSpan(output); long reads_done = 0; long writes_done = 0; long deletes_done = 0; - using var session = store.NewSession(functions); + var di = testLoader.Options.DeleteAndReinsert; + using var session = store.NewSession(functions); var bContext = session.BasicContext; while (!done) @@ -256,27 +263,35 @@ private void RunYcsbSafeContext(int thread_idx) bContext.CompletePending(false); } - int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 - if (r < readPercent) - { - bContext.Read(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _input, ref _output, Empty.Default); - ++reads_done; - continue; - } - if (r < upsertPercent) - { - bContext.Upsert(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _value, Empty.Default); - ++writes_done; - continue; - } - if (r < rmwPercent) + unsafe { - bContext.RMW(ref SpanByte.Reinterpret(ref txn_keys_[idx]), ref _input, Empty.Default); - ++writes_done; - continue; + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + var key = txn_keys_[idx].AsReadOnlySpan(); + + int r = (int)rng.Generate(100); // rng.Next() is not inclusive of the upper bound so this will be <= 99 + if (r < readPercent) + { + bContext.Read(key, ref pinnedInputSpan, ref _output, Empty.Default); + ++reads_done; + continue; + } + if (r < upsertPercent) + { + bContext.Upsert(key, value, Empty.Default); + ++writes_done; + continue; + } + if (r < rmwPercent) + { + bContext.RMW(key, ref pinnedInputSpan, Empty.Default); + ++writes_done; + continue; + } + bContext.Delete(key, Empty.Default); + if (di) + bContext.Upsert(key, value, Empty.Default); + ++deletes_done; } - bContext.Delete(ref SpanByte.Reinterpret(ref txn_keys_[idx]), Empty.Default); - ++deletes_done; } } @@ -412,12 +427,11 @@ private void SetupYcsbUnsafeContext(int thread_idx) } waiter.Wait(); - using var session = store.NewSession(functions); + using var session = store.NewSession(functions); var uContext = session.UnsafeContext; uContext.BeginUnsafe(); - Span value = stackalloc byte[kValueSize]; - ref SpanByte _value = ref SpanByte.Reinterpret(value); + Span value = stackalloc byte[kValueDataSize]; try { @@ -430,14 +444,12 @@ private void SetupYcsbUnsafeContext(int thread_idx) if (idx % 256 == 0) { uContext.Refresh(); - if (idx % 65536 == 0) - { uContext.CompletePending(false); - } } - uContext.Upsert(ref SpanByte.Reinterpret(ref init_keys_[idx]), ref _value, Empty.Default); + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + uContext.Upsert(init_keys_[idx].AsReadOnlySpan(), value, Empty.Default); } } uContext.CompletePending(true); @@ -459,11 +471,10 @@ private void SetupYcsbSafeContext(int thread_idx) } waiter.Wait(); - using var session = store.NewSession(functions); + using var session = store.NewSession(functions); var bContext = session.BasicContext; - Span value = stackalloc byte[kValueSize]; - ref SpanByte _value = ref SpanByte.Reinterpret(value); + Span value = stackalloc byte[kValueDataSize]; for (long chunk_idx = Interlocked.Add(ref idx_, YcsbConstants.kChunkSize) - YcsbConstants.kChunkSize; chunk_idx < InitCount; @@ -474,14 +485,12 @@ private void SetupYcsbSafeContext(int thread_idx) if (idx % 256 == 0) { bContext.Refresh(); - if (idx % 65536 == 0) - { bContext.CompletePending(false); - } } - bContext.Upsert(ref SpanByte.Reinterpret(ref init_keys_[idx]), ref _value, Empty.Default); + // The key vectors are not pinned, but we use only (ReadOnly)Span operations in SessionSpanByteFunctions and key compare. + bContext.Upsert(init_keys_[idx].AsReadOnlySpan(), value, Empty.Default); } } @@ -501,11 +510,7 @@ internal static void CreateKeyVectors(TestLoader testLoader, out KeySpanByte[] i internal class KeySetter : IKeySetter { - public unsafe void Set(KeySpanByte[] vector, long idx, long value) - { - vector[idx].length = kKeySize - 4; - vector[idx].value = value; - } + public void Set(KeySpanByte[] vector, long idx, long value) => vector[idx].value = value; } #endregion diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs index 1b7da927d8b..995bcf4811c 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/TestLoader.cs @@ -22,8 +22,8 @@ class TestLoader { internal readonly Options Options; internal readonly string Distribution; - internal Key[] init_keys = default; - internal Key[] txn_keys = default; + internal FixedLengthKey[] init_keys = default; + internal FixedLengthKey[] txn_keys = default; internal KeySpanByte[] init_span_keys = default; internal KeySpanByte[] txn_span_keys = default; @@ -77,6 +77,16 @@ static bool verifyOption(bool isValid, string name, string info = null) if (!verifyOption(rumdPercents.Length == 4 && Options.RumdPercents.Sum() == 100 && !Options.RumdPercents.Any(x => x < 0), "rmud", "Percentages of [(r)eads,(u)pserts,r(m)ws,(d)eletes] must be empty or must sum to 100 with no negative elements")) return; + if (Options.UseOverflowValues && Options.UseObjectValues) + { + Console.WriteLine($"Cannot specify both UseOverflowValues and UseObjectValues"); + return; + } + if ((Options.UseOverflowValues || Options.UseObjectValues) && BenchmarkType != BenchmarkType.Object) + { + Console.WriteLine($"Can only specify UseOverflowValues or UseObjectValues with BenchmarkType.Object"); + return; + } ReadPercent = rumdPercents[0]; UpsertPercent = ReadPercent + rumdPercents[1]; RmwPercent = UpsertPercent + rumdPercents[2]; @@ -106,15 +116,19 @@ private void LoadDataThreadProc() switch (BenchmarkType) { - case BenchmarkType.Ycsb: - Tsavorite_YcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); - LoadData(this, init_keys, txn_keys, new Tsavorite_YcsbBenchmark.KeySetter()); + case BenchmarkType.FixedLen: + FixedLenYcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); + LoadData(this, init_keys, txn_keys, new FixedLenYcsbBenchmark.KeySetter()); 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()); break; - case BenchmarkType.ConcurrentDictionaryYcsb: + case BenchmarkType.Object: + ObjectYcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); + LoadData(this, init_keys, txn_keys, new ObjectYcsbBenchmark.KeySetter()); + break; + case BenchmarkType.ConcurrentDictionary: ConcurrentDictionary_YcsbBenchmark.CreateKeyVectors(this, out init_keys, out txn_keys); LoadData(this, init_keys, txn_keys, new ConcurrentDictionary_YcsbBenchmark.KeySetter()); break; @@ -340,9 +354,9 @@ private static void LoadSyntheticData(string distribution, uin internal string BackupPath => $"{DataPath}/{Distribution}_{(Options.UseSyntheticData ? "synthetic" : "ycsb")}_{(Options.UseSmallData ? "2.5M_10M" : "250M_1000M")}"; - internal bool MaybeRecoverStore(TsavoriteKV store) - where SF : IStoreFunctions - where A : IAllocator + internal bool MaybeRecoverStore(TsavoriteKV store) + where SF : IStoreFunctions + where A : IAllocator { // Recover database for fast benchmark repeat runs. if (RecoverMode) @@ -371,9 +385,9 @@ internal bool MaybeRecoverStore(TsavoriteKV store) return false; } - internal void MaybeCheckpointStore(TsavoriteKV store) - where SF : IStoreFunctions - where A : IAllocator + internal void MaybeCheckpointStore(TsavoriteKV store) + where SF : IStoreFunctions + where A : IAllocator { // Checkpoint database for fast benchmark repeat runs. if (RecoverMode) diff --git a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/YcsbConstants.cs b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/YcsbConstants.cs index 15da29c3e2c..89334a79fe0 100644 --- a/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/YcsbConstants.cs +++ b/libs/storage/Tsavorite/cs/benchmark/YCSB.benchmark/YcsbConstants.cs @@ -7,9 +7,10 @@ namespace Tsavorite.benchmark { enum BenchmarkType : byte { - Ycsb = 0, + FixedLen = 0, SpanByte, - ConcurrentDictionaryYcsb + Object, + ConcurrentDictionary }; enum AddressLineNum : int diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs index fdc7a946daf..4490e59714d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorBase.cs @@ -14,9 +14,9 @@ namespace Tsavorite.core /// /// 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 - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public abstract partial class AllocatorBase : IDisposable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// The epoch we are operating with protected readonly LightEpoch epoch; @@ -24,11 +24,14 @@ public abstract partial class AllocatorBaseThe store functions for this instance of TsavoriteKV - internal readonly TStoreFunctions _storeFunctions; + internal readonly TStoreFunctions storeFunctions; /// The fully-derived allocator struct wrapper (so calls on it are inlined rather than virtual) for this log. internal readonly TAllocator _wrapper; + /// Sometimes it's useful to know this explicitly rather than rely on method overrides etc. + internal bool IsObjectAllocator = false; + #region Protected size definitions /// Buffer size internal readonly int BufferSize; @@ -167,13 +170,13 @@ public override string ToString() private readonly ErrorList errorList = new(); /// Observer for records entering read-only region - internal IObserver> OnReadOnlyObserver; + internal IObserver OnReadOnlyObserver; /// Observer for records getting evicted from memory (page closed) - internal IObserver> OnEvictionObserver; + internal IObserver OnEvictionObserver; /// Observer for records brought into memory by deserializing pages - internal IObserver> OnDeserializationObserver; + internal IObserver OnDeserializationObserver; /// The "event" to be waited on for flush completion by the initiator of an operation internal CompletionEvent FlushEvent; @@ -200,7 +203,7 @@ public override string ToString() 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 unsafe void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default); /// 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); @@ -269,8 +272,9 @@ void FlushRunner() while (physicalAddress < endPhysicalAddress) { - ref var info = ref _wrapper.GetInfo(physicalAddress); - var (_, alignedRecordSize) = _wrapper.GetRecordSize(physicalAddress); + var logRecord = _wrapper.CreateLogRecord(logicalAddress); + ref var info = ref logRecord.InfoRef; + var (_, alignedRecordSize) = logRecord.GetInlineRecordSizes(); if (info.Dirty) { info.ClearDirtyAtomic(); // there may be read locks being taken, hence atomic @@ -313,9 +317,6 @@ void FlushRunner() } } - /// Delete in-memory portion of the log - internal abstract void DeleteFromMemory(); - /// Reset the hybrid log. WARNING: assumes that threads have drained out at this point. public virtual void Reset() { @@ -386,6 +387,38 @@ public virtual void Dispose() #endregion abstract and virtual methods + #region LogRecord functions + + /// Get start logical address + public long GetStartLogicalAddress(long page) => page << LogPageSizeBits; + + /// Get first valid address + public long GetFirstValidLogicalAddress(long page) => page == 0 ? Constants.kFirstValidAddress : page << LogPageSizeBits; + + public static unsafe ref RecordInfo GetInfoFromBytePointer(byte* ptr) => ref Unsafe.AsRef(ptr); + + [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) @@ -415,10 +448,11 @@ internal unsafe void ApplyDelta(DeltaLog log, long startPage, long endPage, long physicalAddress += sizeof(int); if (address >= startLogicalAddress && address < endLogicalAddress) { - var destination = _wrapper.GetPhysicalAddress(address); + var logRecord = _wrapper.CreateLogRecord(address); + var destination = logRecord.physicalAddress; // Clear extra space (if any) in old record - var oldSize = _wrapper.GetRecordSize(destination).Item2; + var oldSize = logRecord.GetInlineRecordSizes().allocatedSize; if (oldSize > size) new Span((byte*)(destination + size), oldSize - size).Clear(); @@ -426,7 +460,7 @@ internal unsafe void ApplyDelta(DeltaLog log, long startPage, long endPage, long Buffer.MemoryCopy((void*)physicalAddress, (void*)destination, size, size); // Clean up temporary bits when applying the delta log - ref var destInfo = ref _wrapper.GetInfo(destination); + ref var destInfo = ref LogRecord.GetInfoRef(destination); destInfo.ClearBitsForDiskImages(); } physicalAddress += size; @@ -480,7 +514,7 @@ internal void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDes if (asyncResult.partial) { // Write only required bytes within the page - int aligned_start = (int)((asyncResult.fromAddress - (asyncResult.page << LogPageSizeBits))); + int aligned_start = (int)(asyncResult.fromAddress - (asyncResult.page << LogPageSizeBits)); aligned_start = (aligned_start / sectorSize) * sectorSize; int aligned_end = (int)(asyncResult.untilAddress - (asyncResult.page << LogPageSizeBits)); @@ -503,7 +537,7 @@ internal void WriteAsync(IntPtr alignedSourceAddress, ulong alignedDes /// Instantiate base allocator implementation private protected AllocatorBase(LogSettings settings, TStoreFunctions storeFunctions, Func wrapperCreator, Action evictCallback, LightEpoch epoch, Action flushCallback, ILogger logger = null) { - _storeFunctions = storeFunctions; + this.storeFunctions = storeFunctions; _wrapper = wrapperCreator(this); // Validation @@ -526,6 +560,11 @@ 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}"); + this.logger = logger; if (settings.LogDevice == null) throw new TsavoriteException("LogSettings.LogDevice needs to be specified (e.g., use Devices.CreateLogDevice, AzureStorageDevice, or NullDevice)"); @@ -1141,7 +1180,7 @@ private void OnPagesMarkedReadOnly(long newSafeReadOnlyAddress, bool noFlush = f { // This scan does not need a store because it does not lock; it is epoch-protected so by the time it runs no current thread // will have seen a record below the new ReadOnlyAddress as "in mutable region". - using var iter = Scan(store: null, oldSafeReadOnlyAddress, newSafeReadOnlyAddress, ScanBufferingMode.NoBuffering); + using var iter = Scan(store: null, oldSafeReadOnlyAddress, newSafeReadOnlyAddress, DiskScanBufferingMode.NoBuffering); OnReadOnlyObserver?.OnNext(iter); } AsyncFlushPages(oldSafeReadOnlyAddress, newSafeReadOnlyAddress, noFlush); @@ -1403,7 +1442,7 @@ public void RecoveryReset(long tailAddress, long headAddress, long beginAddress, } /// Invoked by users to obtain a record from disk. It uses sector aligned memory to read the record efficiently into memory. - internal unsafe void AsyncReadRecordToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, ref AsyncIOContext context) + internal unsafe void AsyncReadRecordToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, ref AsyncIOContext context) { var fileOffset = (ulong)(AlignedPageSizeBytes * (fromLogical >> LogPageSizeBits) + (fromLogical & PageSizeMask)); var alignedFileOffset = (ulong)(((long)fileOffset / sectorSize) * sectorSize); @@ -1416,7 +1455,7 @@ internal unsafe void AsyncReadRecordToMemory(long fromLogical, int numBytes, Dev record.available_bytes = (int)(alignedReadLength - (fileOffset - alignedFileOffset)); record.required_bytes = numBytes; - var asyncResult = default(AsyncGetFromDiskResult>); + var asyncResult = default(AsyncGetFromDiskResult); asyncResult.context = context; asyncResult.context.record = record; device.ReadAsync(alignedFileOffset, @@ -1549,6 +1588,8 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = fromAddress = pageStartAddress, untilAddress = pageEndAddress }; + + // If either fromAddress or untilAddress is in the middle of the page, prepare to handle a partial page if ( ((fromAddress > pageStartAddress) && (fromAddress < pageEndAddress)) || ((untilAddress > pageStartAddress) && (untilAddress < pageEndAddress)) @@ -1580,10 +1621,10 @@ public void AsyncFlushPages(long fromAddress, long untilAddress, bool noFlush = skip = true; } - if (skip) continue; + if (skip) + continue; - // Partial page starting point, need to wait until the - // ongoing adjacent flush is completed to ensure correctness + // Partial page starting point, need to wait until the ongoing adjacent flush is completed to ensure correctness if (GetOffsetInPage(asyncResult.fromAddress) > 0) { var index = GetPageIndexForAddress(asyncResult.fromAddress); @@ -1690,7 +1731,7 @@ void FlushRunner() } } - internal void AsyncGetFromDisk(long fromLogical, int numBytes, AsyncIOContext context, SectorAlignedMemory result = default) + internal void AsyncGetFromDisk(long fromLogical, int numBytes, AsyncIOContext context, SectorAlignedMemory result = default) { if (epoch.ThisInstanceProtected()) // Do not spin for unprotected IO threads { @@ -1708,30 +1749,77 @@ internal void AsyncGetFromDisk(long fromLogical, int numBytes, AsyncIOContext + /// Read pages from specified device + /// + internal void AsyncReadPagesFromDeviceToFrame( + long readPageStart, + int numPages, + long untilAddress, + DeviceIOCompletionCallback callback, + TContext context, + BlittableFrame frame, + out CountdownEvent completed, + long devicePageOffset = 0, + IDevice device = null, IDevice objectLogDevice = null) + { + var usedDevice = device ?? this.device; + + completed = new CountdownEvent(numPages); + for (long readPage = readPageStart; readPage < (readPageStart + numPages); readPage++) + { + int pageIndex = (int)(readPage % frame.frameSize); + if (frame.frame[pageIndex] == null) + frame.Allocate(pageIndex); + else + frame.Clear(pageIndex); + + var asyncResult = new PageAsyncReadResult() + { + page = readPage, + context = context, + handle = completed, + frame = frame + }; + + ulong offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); + uint readLength = (uint)AlignedPageSizeBytes; + long adjustedUntilAddress = (AlignedPageSizeBytes * (untilAddress >> LogPageSizeBits) + (untilAddress & PageSizeMask)); + + if (adjustedUntilAddress > 0 && ((adjustedUntilAddress - (long)offsetInFile) < PageSize)) + { + readLength = (uint)(adjustedUntilAddress - (long)offsetInFile); + readLength = (uint)((readLength + (sectorSize - 1)) & ~(sectorSize - 1)); + } + + if (device != null) + offsetInFile = (ulong)(AlignedPageSizeBytes * (readPage - devicePageOffset)); + + usedDevice.ReadAsync(offsetInFile, (IntPtr)frame.pointers[pageIndex], readLength, callback, asyncResult); + } + } + private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, object context) { if (errorCode != 0) logger?.LogError("AsyncGetFromDiskCallback error: {0}", errorCode); - var result = (AsyncGetFromDiskResult>)context; + var result = (AsyncGetFromDiskResult)context; var ctx = result.context; try { - var record = ctx.record.GetValidPointer(); - int requiredBytes = _wrapper.GetRequiredRecordSize((long)record, ctx.record.available_bytes); - if (ctx.record.available_bytes >= requiredBytes) + bool hasFullRecord = DiskLogRecord.IsComplete(ctx.record, out bool hasFullKey, out int requiredBytes); // TODO: support 'long' lengths + if (hasFullKey || ctx.request_key.IsEmpty) { - Debug.Assert(!_wrapper.GetInfoFromBytePointer(record).Invalid, "Invalid records should not be in the hash chain for pending IO"); - - // We have all the required bytes. If we don't have the complete record, RetrievedFullRecord calls AsyncGetFromDisk. - if (!_wrapper.RetrievedFullRecord(record, ref ctx)) - return; + 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"); // If request_key is null we're called from ReadAtAddress, so it is an implicit match. - if (ctx.request_key is not null && !_storeFunctions.KeysEqual(ref ctx.request_key.Get(), ref _wrapper.GetContextRecordKey(ref ctx))) + 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 = _wrapper.GetInfoFromBytePointer(record).PreviousAddress; + ctx.logicalAddress = diskLogRecord.Info.PreviousAddress; if (ctx.logicalAddress >= BeginAddress && ctx.logicalAddress >= ctx.minAddress) { ctx.record.Return(); @@ -1741,16 +1829,25 @@ private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, obje } } - // Either the keys 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 if (ctx.callbackQueue is not null) - ctx.callbackQueue.Enqueue(ctx); - else - _ = ctx.asyncOperation.TrySetResult(ctx); + if (hasFullRecord) + { + // Either the keys match or we are below the range to retrieve (which ContinuePending* will detect), so we're done. + if (currentRecordIsInRange) + ctx.ValueObject = diskLogRecord.DeserializeValueObject(storeFunctions.CreateValueObjectSerializer()); + + if (ctx.completionEvent is not null) + ctx.completionEvent.Set(ref ctx); + else + ctx.callbackQueue.Enqueue(ctx); + } } - else + + 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); } @@ -1760,8 +1857,6 @@ private unsafe void AsyncGetFromDiskCallback(uint errorCode, uint numBytes, obje logger?.LogError(e, "AsyncGetFromDiskCallback error"); if (ctx.completionEvent is not null) ctx.completionEvent.SetException(e); - else if (ctx.asyncOperation is not null) - _ = ctx.asyncOperation.TrySetException(e); else throw; } @@ -1873,8 +1968,9 @@ protected void AsyncFlushPageToDeviceCallback(uint errorCode, uint numBytes, obj while (physicalAddress < endPhysicalAddress) { - ref var info = ref _wrapper.GetInfo(physicalAddress); - var (_, alignedRecordSize) = _wrapper.GetRecordSize(physicalAddress); + var logRecord = _wrapper.CreateLogRecord(startAddress); + ref var info = ref logRecord.InfoRef; + var (_, alignedRecordSize) = logRecord.GetInlineRecordSizes(); if (info.Dirty) info.ClearDirtyAtomic(); // there may be read locks being taken, hence atomic physicalAddress += alignedRecordSize; diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorRecord.cs deleted file mode 100644 index 17543c839e0..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorRecord.cs +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Runtime.InteropServices; - -#pragma warning disable CS1591 // Missing XML comment for publicly visible type or member - -namespace Tsavorite.core -{ - [StructLayout(LayoutKind.Sequential, Pack = 1)] - public struct AllocatorRecord - { - public RecordInfo info; - public TKey key; - public TValue value; - - public override string ToString() - { - var keyString = key?.ToString() ?? "null"; - if (keyString.Length > 20) - keyString = keyString.Substring(0, 20) + "..."; - var valueString = value?.ToString() ?? "null"; ; - if (valueString.Length > 20) - valueString = valueString.Substring(0, 20) + "..."; - return $"{keyString} | {valueString} | {info}"; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs index e137bcd9aa7..f8dc62c628f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorScan.cs @@ -8,32 +8,32 @@ namespace Tsavorite.core { - public abstract partial class AllocatorBase : IDisposable - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public abstract partial class AllocatorBase : IDisposable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Pull-based scan interface for HLOG; user calls GetNext() which advances through the address range. /// /// Pull Scan iterator instance - public abstract ITsavoriteScanIterator Scan(TsavoriteKV store, long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering, bool includeSealedRecords = false); + public abstract ITsavoriteScanIterator Scan(TsavoriteKV store, long beginAddress, long endAddress, DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering, bool includeSealedRecords = false); /// /// Push-based scan interface for HLOG, called from LogAccessor; scan the log given address range, calling for each record. /// /// True if Scan completed; false if Scan ended early due to one of the TScanIterator reader functions returning false - internal abstract bool Scan(TsavoriteKV store, long beginAddress, long endAddress, ref TScanFunctions scanFunctions, - ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering) - where TScanFunctions : IScanIteratorFunctions; + internal abstract bool Scan(TsavoriteKV store, long beginAddress, long endAddress, ref TScanFunctions scanFunctions, + DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering) + where TScanFunctions : IScanIteratorFunctions; /// /// Push-based iteration of key versions, calling for each record. /// /// True if Scan completed; false if Scan ended early due to one of the TScanIterator reader functions returning false - internal bool IterateKeyVersions(TsavoriteKV store, ref TKey key, ref TScanFunctions scanFunctions) - where TScanFunctions : IScanIteratorFunctions + internal bool IterateKeyVersions(TsavoriteKV store, ReadOnlySpan key, ref TScanFunctions scanFunctions) + where TScanFunctions : IScanIteratorFunctions { - OperationStackContext stackCtx = new(_storeFunctions.GetKeyHashCode64(ref key)); + OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(key)); if (!store.FindTag(ref stackCtx.hei)) return false; stackCtx.SetRecordSourceToHashEntry(store.hlogBase); @@ -41,22 +41,22 @@ internal bool IterateKeyVersions(TsavoriteKV /// Push-based iteration of key versions, calling for each record. /// /// True if Scan completed; false if Scan ended early due to one of the TScanIterator reader functions returning false - internal abstract bool IterateKeyVersions(TsavoriteKV store, ref TKey key, long beginAddress, ref TScanFunctions scanFunctions) - where TScanFunctions : IScanIteratorFunctions; + internal abstract bool IterateKeyVersions(TsavoriteKV store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) + where TScanFunctions : IScanIteratorFunctions; /// /// Implementation for push-scanning Tsavorite log /// internal bool PushScanImpl(long beginAddress, long endAddress, ref TScanFunctions scanFunctions, TScanIterator iter) - where TScanFunctions : IScanIteratorFunctions - where TScanIterator : ITsavoriteScanIterator, IPushScanIterator + where TScanFunctions : IScanIteratorFunctions + where TScanIterator : ITsavoriteScanIterator, IPushScanIterator { if (!scanFunctions.OnStart(beginAddress, endAddress)) return false; @@ -64,18 +64,15 @@ internal bool PushScanImpl(long beginAddress, lon long numRecords = 1; var stop = false; - for (; !stop && iter.GetNext(out var recordInfo); ++numRecords) + for (; !stop && iter.GetNext(); ++numRecords) { try { - // Pull Iter records are in temp storage so do not need locks, but we'll call ConcurrentReader because, for example, GenericAllocator - // may need to know the object is in that region. - if (recordInfo.IsClosed) // Iterator checks this but it may have changed since + if (iter.Info.IsClosed) // Iterator checks this but it may have changed since continue; - if (iter.CurrentAddress >= headAddress) - stop = !scanFunctions.ConcurrentReader(ref iter.GetKey(), ref iter.GetValue(), new RecordMetadata(recordInfo, iter.CurrentAddress), numRecords, out _); - else - stop = !scanFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), new RecordMetadata(recordInfo, iter.CurrentAddress), numRecords, out _); + + // Pull Iter records are in temp storage so do not need locks. + stop = !scanFunctions.Reader(ref iter, new RecordMetadata(iter.CurrentAddress), numRecords, out _); } catch (Exception ex) { @@ -91,29 +88,26 @@ internal bool PushScanImpl(long beginAddress, lon /// /// Implementation for push-iterating key versions /// - internal bool IterateKeyVersionsImpl(TsavoriteKV store, ref TKey key, long beginAddress, ref TScanFunctions scanFunctions, TScanIterator iter) - where TScanFunctions : IScanIteratorFunctions - where TScanIterator : ITsavoriteScanIterator, IPushScanIterator + internal bool IterateHashChain(TsavoriteKV store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions, TScanIterator iter) + where TScanFunctions : IScanIteratorFunctions + where TScanIterator : ITsavoriteScanIterator, IPushScanIterator { if (!scanFunctions.OnStart(beginAddress, Constants.kInvalidAddress)) return false; - var headAddress = HeadAddress; + var readOnlyAddress = ReadOnlyAddress; long numRecords = 1; bool stop = false, continueOnDisk = false; - for (; !stop && iter.BeginGetPrevInMemory(ref key, out var recordInfo, out continueOnDisk); ++numRecords) + for (; !stop && iter.BeginGetPrevInMemory(key, out var logRecord, out continueOnDisk); ++numRecords) { - OperationStackContext stackCtx = default; + OperationStackContext stackCtx = default; try { - // Iter records above headAddress will be in log memory and must be locked. - if (iter.CurrentAddress >= headAddress && !recordInfo.IsClosed) - { - store.LockForScan(ref stackCtx, ref key); - stop = !scanFunctions.ConcurrentReader(ref key, ref iter.GetValue(), new RecordMetadata(recordInfo, iter.CurrentAddress), numRecords, out _); - } - else - stop = !scanFunctions.SingleReader(ref key, ref iter.GetValue(), new RecordMetadata(recordInfo, iter.CurrentAddress), numRecords, out _); + // Iter records above readOnlyAddress will be in mutable log memory so the chain must be locked. + // We hold the epoch so iter does not need to copy, so do not use iter's ISourceLogRecord implementation; create a local LogRecord around the address. + if (iter.CurrentAddress >= readOnlyAddress && !logRecord.Info.IsClosed) + store.LockForScan(ref stackCtx, key); + stop = !scanFunctions.Reader(ref logRecord, new RecordMetadata(iter.CurrentAddress), numRecords, out _); } catch (Exception ex) { @@ -130,11 +124,11 @@ internal bool IterateKeyVersionsImpl(TsavoriteKV< if (continueOnDisk) { - AsyncIOContextCompletionEvent completionEvent = new(); + AsyncIOContextCompletionEvent completionEvent = new(); try { var logicalAddress = iter.CurrentAddress; - while (!stop && GetFromDiskAndPushToReader(ref key, ref logicalAddress, ref scanFunctions, numRecords, completionEvent, out stop)) + while (!stop && GetFromDiskAndPushToReader(key, ref logicalAddress, ref scanFunctions, numRecords, completionEvent, out stop)) ++numRecords; } catch (Exception ex) @@ -152,13 +146,13 @@ internal bool IterateKeyVersionsImpl(TsavoriteKV< return !stop; } - internal unsafe bool GetFromDiskAndPushToReader(ref TKey key, ref long logicalAddress, ref TScanFunctions scanFunctions, long numRecords, - AsyncIOContextCompletionEvent completionEvent, out bool stop) - where TScanFunctions : IScanIteratorFunctions + internal unsafe bool GetFromDiskAndPushToReader(ReadOnlySpan key, ref long logicalAddress, ref TScanFunctions scanFunctions, long numRecords, + AsyncIOContextCompletionEvent completionEvent, out bool stop) + where TScanFunctions : IScanIteratorFunctions { - completionEvent.Prepare(_wrapper.GetKeyContainer(ref key), logicalAddress); + completionEvent.Prepare(PinnedSpanByte.FromPinnedSpan(key), logicalAddress); - AsyncGetFromDisk(logicalAddress, _wrapper.GetAverageRecordSize(), completionEvent.request); + AsyncGetFromDisk(logicalAddress, DiskLogRecord.InitialIOSize, completionEvent.request); completionEvent.Wait(); stop = false; @@ -170,10 +164,10 @@ internal unsafe bool GetFromDiskAndPushToReader(ref TKey key, re if (completionEvent.request.logicalAddress < BeginAddress) return false; - RecordInfo recordInfo = _wrapper.GetInfoFromBytePointer(completionEvent.request.record.GetValidPointer()); - recordInfo.ClearBitsForDiskImages(); - stop = !scanFunctions.SingleReader(ref key, ref _wrapper.GetContextRecordValue(ref completionEvent.request), new RecordMetadata(recordInfo, completionEvent.request.logicalAddress), numRecords, out _); - logicalAddress = recordInfo.PreviousAddress; + var logRecord = new DiskLogRecord(ref completionEvent.request); + logRecord.InfoRef.ClearBitsForDiskImages(); + stop = !scanFunctions.Reader(ref logRecord, new RecordMetadata(completionEvent.request.logicalAddress), numRecords, out _); + logicalAddress = logRecord.Info.PreviousAddress; return !stop; } @@ -184,23 +178,23 @@ internal unsafe bool GetFromDiskAndPushToReader(ref TKey key, re /// True if Scan completed and pushed records; false if Scan ended early due to finding less than records /// or one of the TScanIterator reader functions returning false /// Currently we load an entire page, which while inefficient in performance, allows us to make the cursor safe (by ensuring we align to a valid record) if it is not - /// the last one returned. We could optimize this to load only the subset of a page that is pointed to by the cursor and do GetRequiredRecordSize/RetrievedFullRecord as in + /// the last one returned. We could optimize this to load only the subset of a page that is pointed to by the cursor and use DiskLogRecord.GetSerializedRecordLength as in /// AsyncGetFromDiskCallback. However, this would not validate the cursor and would therefore require maintaining a cursor history. - internal abstract bool ScanCursor(TsavoriteKV store, ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) - where TScanFunctions : IScanIteratorFunctions; + internal abstract bool ScanCursor(TsavoriteKV store, ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) + where TScanFunctions : IScanIteratorFunctions; - private protected bool ScanLookup(TsavoriteKV store, - ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, TScanIterator iter, bool validateCursor, long maxAddress) - where TScanFunctions : IScanIteratorFunctions - where TScanIterator : ITsavoriteScanIterator, IPushScanIterator + private protected bool ScanLookup(TsavoriteKV store, + ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, TScanIterator iter, bool validateCursor, long maxAddress) + where TScanFunctions : IScanIteratorFunctions + where TScanIterator : ITsavoriteScanIterator, IPushScanIterator { - using var session = store.NewSession>(new LogScanCursorFunctions()); + using var session = store.NewSession>(new NoOpSessionFunctions()); var bContext = session.BasicContext; if (cursor < BeginAddress) // This includes 0, which means to start the Scan cursor = BeginAddress; - else if (validateCursor) - iter.SnapCursorToLogicalAddress(ref cursor); + else if (validateCursor && !iter.SnapCursorToLogicalAddress(ref cursor)) + goto IterationComplete; if (!scanFunctions.OnStart(cursor, iter.EndAddress)) return false; @@ -211,19 +205,17 @@ private protected bool ScanLookup 256) { - bContext.CompletePending(wait: true); + _ = bContext.CompletePending(wait: true); numPending = 0; } } @@ -247,7 +239,7 @@ private protected bool ScanLookup 0) - bContext.CompletePending(wait: true); + _ = bContext.CompletePending(wait: true); IterationComplete: cursor = 0; @@ -256,46 +248,43 @@ private protected bool ScanLookup(TSessionFunctionsWrapper sessionFunctions, ScanCursorState scanCursorState, RecordInfo recordInfo, - ref TKey key, ref TValue value, long currentAddress, long minAddress, long maxAddress) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ConditionalScanPush(TSessionFunctionsWrapper sessionFunctions, + ScanCursorState scanCursorState, ref TSourceLogRecord srcLogRecord, long currentAddress, long minAddress, long maxAddress) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { Debug.Assert(epoch.ThisInstanceProtected(), "This is called only from ScanLookup so the epoch should be protected"); - TsavoriteKV.PendingContext pendingContext = new(_storeFunctions.GetKeyHashCode64(ref key)); + TsavoriteKV.PendingContext pendingContext = new(storeFunctions.GetKeyHashCode64(srcLogRecord.Key)); OperationStatus internalStatus; - OperationStackContext stackCtx = new(pendingContext.keyHash); + OperationStackContext stackCtx = new(pendingContext.keyHash); bool needIO; do { // If a more recent version of the record exists, do not push this one. Start by searching in-memory. - if (sessionFunctions.Store.TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx, currentAddress, minAddress, maxAddress, out internalStatus, out needIO)) + if (sessionFunctions.Store.TryFindRecordInMainLogForConditionalOperation(sessionFunctions, srcLogRecord.Key, ref stackCtx, + currentAddress, minAddress, maxAddress, out internalStatus, out needIO)) return Status.CreateFound(); } while (sessionFunctions.Store.HandleImmediateNonPendingRetryStatus(internalStatus, sessionFunctions)); - TInput input = default; - TOutput output = default; if (needIO) { // A more recent version of the key was not (yet) found and we need another IO to continue searching. - internalStatus = PrepareIOForConditionalScan(sessionFunctions, ref pendingContext, ref key, ref input, ref value, ref output, default, - ref stackCtx, minAddress, maxAddress, scanCursorState); + internalStatus = PrepareIOForConditionalScan(sessionFunctions.Store, ref pendingContext, ref srcLogRecord, ref stackCtx, minAddress, maxAddress, scanCursorState); } else { // A more recent version of the key was not found. recSrc.LogicalAddress is the correct address, because minAddress was examined // and this is the previous record in the tag chain. Push this record to the user. - RecordMetadata recordMetadata = new(recordInfo, stackCtx.recSrc.LogicalAddress); - var stop = (stackCtx.recSrc.LogicalAddress >= HeadAddress) - ? !scanCursorState.functions.ConcurrentReader(ref key, ref value, recordMetadata, scanCursorState.acceptedCount, out var cursorRecordResult) - : !scanCursorState.functions.SingleReader(ref key, ref value, recordMetadata, scanCursorState.acceptedCount, out cursorRecordResult); + RecordMetadata recordMetadata = new(stackCtx.recSrc.LogicalAddress); + var stop = !scanCursorState.functions.Reader(ref srcLogRecord, recordMetadata, scanCursorState.acceptedCount, out var cursorRecordResult); if (stop) scanCursorState.stop = true; else { if ((cursorRecordResult & CursorRecordResult.Accept) != 0) - Interlocked.Increment(ref scanCursorState.acceptedCount); + _ = Interlocked.Increment(ref scanCursorState.acceptedCount); if ((cursorRecordResult & CursorRecordResult.EndBatch) != 0) scanCursorState.endBatch = true; if ((cursorRecordResult & CursorRecordResult.RetryLastRecord) != 0) @@ -307,87 +296,22 @@ internal Status ConditionalScanPush(TSessionFunctionsWrapper sessionFunctions, - ref TsavoriteKV.PendingContext pendingContext, - ref TKey key, ref TInput input, ref TValue value, ref TOutput output, TContext userContext, - ref OperationStackContext stackCtx, long minAddress, long maxAddress, ScanCursorState scanCursorState) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal static OperationStatus PrepareIOForConditionalScan(TsavoriteKV store, + ref TsavoriteKV.PendingContext pendingContext, ref TSourceLogRecord srcLogRecord, + ref OperationStackContext stackCtx, long minAddress, long maxAddress, ScanCursorState scanCursorState) + where TSourceLogRecord : ISourceLogRecord { - // WriteReason is not surfaced for this operation, so pick anything. - var status = sessionFunctions.Store.PrepareIOForConditionalOperation(sessionFunctions, ref pendingContext, ref key, ref input, ref value, ref output, - userContext, ref stackCtx, minAddress, maxAddress, WriteReason.Compaction, OperationType.CONDITIONAL_SCAN_PUSH); + var status = store.PrepareIOForConditionalOperation(ref pendingContext, ref srcLogRecord, ref stackCtx, minAddress, maxAddress, OperationType.CONDITIONAL_SCAN_PUSH); pendingContext.scanCursorState = scanCursorState; return status; } - internal struct LogScanCursorFunctions : ISessionFunctions - { - public bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo) => true; - public bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) => true; - public void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { } - - public bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - public void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo) { } - public bool ConcurrentDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - - public bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) => true; - public void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason) { } - public bool ConcurrentWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => true; - - public bool InPlaceUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - - public bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo) => true; - public bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - public bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo) => true; - - public bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; - public bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - public void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo) { } - - public void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { } - - public int GetRMWModifiedValueLength(ref TValue value, ref TInput input) => 0; - public int GetRMWInitialValueLength(ref TInput input) => 0; - public int GetUpsertValueLength(ref TValue value, ref TInput input) => 0; - - public void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal long SnapToFixedLengthLogicalAddressBoundary(ref long logicalAddress, int recordSize) - { - // Get the initial offset on the page - int offset = (int)(logicalAddress & PageSizeMask); - long pageStart = logicalAddress - offset; - - int recordStartOffset; - if (logicalAddress < PageSize) - { - // We are on the first page so must account for BeginAddress. - if (offset < BeginAddress) - return logicalAddress = BeginAddress; - recordStartOffset = (int)(((offset - BeginAddress) / recordSize) * recordSize + BeginAddress); - } - else - { - // Not the first page, so just find the highest recordStartOffset <= offset. - recordStartOffset = (offset / recordSize) * recordSize; - } - - // If there is not enough room for a full record, advance logicalAddress to the next page start. - if (PageSize - recordStartOffset >= recordSize) - logicalAddress = pageStart + recordStartOffset; - else - logicalAddress = pageStart + PageSize; - return logicalAddress; - } - /// /// Scan page guaranteed to be in memory /// /// Begin address /// End address /// Observer of scan - internal abstract void MemoryPageScan(long beginAddress, long endAddress, IObserver> observer); + internal abstract void MemoryPageScan(long beginAddress, long endAddress, IObserver observer); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorSettings.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorSettings.cs index 935859aa374..82645ec88f7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AllocatorSettings.cs @@ -7,17 +7,17 @@ namespace Tsavorite.core { /// - /// This class is created by to pass parameters to the allocator factory function. + /// This class is created by to pass parameters to the allocator factory function. /// public struct AllocatorSettings { - /// The Log settings, usually from + /// The Log settings, usually from internal LogSettings LogSettings; - /// The epoch created for the + /// The epoch created for the internal LightEpoch epoch; - /// The logger to use, either from or created by + /// The logger to use, either from or created by internal ILogger logger; /// The action to call on page eviction; used only for readcache diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/AsyncIOContext.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/AsyncIOContext.cs index 1794ebe84a1..f54e7d1c8b7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/AsyncIOContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/AsyncIOContext.cs @@ -4,14 +4,13 @@ using System; using System.Runtime.CompilerServices; using System.Threading; -using System.Threading.Tasks; namespace Tsavorite.core { /// /// Async IO context for PMM /// - public unsafe struct AsyncIOContext + public unsafe struct AsyncIOContext { /// /// Id @@ -19,19 +18,14 @@ public unsafe struct AsyncIOContext public long id; /// - /// Key + /// Key; this is a shallow copy of the key in pendingContext, pointing to its diskLogRecord /// - public IHeapContainer request_key; + public PinnedSpanByte request_key; /// - /// Retrieved key + /// Deserialized ValueObject if RecordInfo.ValueIsObject, else null /// - public TKey key; - - /// - /// Retrieved value - /// - public TValue value; + public IHeapObject ValueObject; /// /// Logical address @@ -56,22 +50,17 @@ public unsafe struct AsyncIOContext /// /// Callback queue /// - public AsyncQueue> callbackQueue; - - /// - /// Async Operation ValueTask backer - /// - public TaskCompletionSource> asyncOperation; + public AsyncQueue callbackQueue; /// /// Synchronous completion event /// - internal AsyncIOContextCompletionEvent completionEvent; + internal AsyncIOContextCompletionEvent completionEvent; /// /// Indicates whether this is a default instance with no pending operation /// - public bool IsDefault() => callbackQueue is null && asyncOperation is null && completionEvent is null; + public readonly bool IsDefault() => callbackQueue is null && completionEvent is null; /// /// Dispose @@ -85,11 +74,11 @@ record = null; } // Wrapper class so we can communicate back the context.record even if it has to retry due to incomplete records. - internal sealed class AsyncIOContextCompletionEvent : IDisposable + internal sealed class AsyncIOContextCompletionEvent : IDisposable { internal SemaphoreSlim semaphore; internal Exception exception; - internal AsyncIOContext request; + internal AsyncIOContext request; internal AsyncIOContextCompletionEvent() { @@ -99,7 +88,13 @@ internal AsyncIOContextCompletionEvent() request.completionEvent = this; } - internal void Prepare(IHeapContainer request_key, long logicalAddress) + /// + /// Prepares to issue an async IO. + /// + /// + /// SAFETY: The MUST be non-movable, such as on the stack, or pinned for the life of the IO operation. + /// + internal void Prepare(PinnedSpanByte request_key, long logicalAddress) { request.Dispose(); request.request_key = request_key; @@ -107,12 +102,12 @@ internal void Prepare(IHeapContainer request_key, long logicalAddress) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Set(ref AsyncIOContext ctx) + internal void Set(ref AsyncIOContext ctx) { request.Dispose(); request = ctx; exception = null; - semaphore.Release(1); + _ = semaphore.Release(1); } [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -121,7 +116,7 @@ internal void SetException(Exception ex) request.Dispose(); request = default; exception = ex; - semaphore.Release(1); + _ = semaphore.Release(1); } internal void Wait(CancellationToken token = default) => semaphore.Wait(token); diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocator.cs deleted file mode 100644 index 54f48093764..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocator.cs +++ /dev/null @@ -1,189 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Runtime.CompilerServices; - -namespace Tsavorite.core -{ - /// - /// Struct wrapper (for inlining) around the fixed-length Blittable allocator. - /// - public struct BlittableAllocator : IAllocator - where TStoreFunctions : IStoreFunctions - { - /// The wrapped class containing all data and most actual functionality. This must be the ONLY field in this structure so its size is sizeof(IntPtr). - private readonly BlittableAllocatorImpl _this; - - public BlittableAllocator(AllocatorSettings settings, TStoreFunctions storeFunctions) - { - // Called by TsavoriteKV via allocatorCreator; must pass a wrapperCreator to AllocatorBase - _this = new(settings, storeFunctions, @this => new BlittableAllocator(@this)); - } - - public BlittableAllocator(object @this) - { - // Called by AllocatorBase via primary ctor wrapperCreator - _this = (BlittableAllocatorImpl)@this; - } - - /// - public readonly AllocatorBase GetBase() - where TAllocator : IAllocator - => (AllocatorBase)(object)_this; - - /// - public readonly bool IsFixedLength => true; - - /// - public readonly bool HasObjectLog => false; - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetStartLogicalAddress(long page) => _this.GetStartLogicalAddress(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddress(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref RecordInfo GetInfo(long physicalAddress) - => ref BlittableAllocatorImpl.GetInfo(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe ref RecordInfo GetInfoFromBytePointer(byte* ptr) - => ref BlittableAllocatorImpl.GetInfoFromBytePointer(ptr); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TKey GetKey(long physicalAddress) - => ref BlittableAllocatorImpl.GetKey(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TValue GetValue(long physicalAddress) - => ref BlittableAllocatorImpl.GetValue(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TValue GetAndInitializeValue(long physicalAddress, long endPhysicalAddress) => ref GetValue(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) - => BlittableAllocatorImpl.GetRecordSize(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref TKey key, ref TInput input, ref TValue value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - where TVariableLengthInput : IVariableLengthInput - => BlittableAllocatorImpl.GetRMWCopyDestinationRecordSize(ref key, ref input, ref value, ref recordInfo, varlenInput); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref TKey key) - => BlittableAllocatorImpl.GetTombstoneRecordSize(ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetRequiredRecordSize(long physicalAddress, int availableBytes) => GetAverageRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetAverageRecordSize() - => BlittableAllocatorImpl.GetAverageRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetFixedRecordSize() - => BlittableAllocatorImpl.GetFixedRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref TKey key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => BlittableAllocatorImpl.GetRMWInitialRecordSize(ref key, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref TKey key, ref TValue value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => BlittableAllocatorImpl.GetUpsertRecordSize(ref key, ref value, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref TKey key, ref TValue value) - => BlittableAllocatorImpl.GetRecordSize(ref key, ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetValueLength(ref TValue value) - => BlittableAllocatorImpl.GetValueLength(ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) - => BlittableAllocatorImpl.RetrievedFullRecord(record, ref ctx); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void AllocatePage(int pageIndex) => _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) - => BlittableAllocatorImpl.PopulatePage(src, required_bytes, destinationPageIndex); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void MarkPage(long logicalAddress, long version) => _this.MarkPage(logicalAddress, version); - - /// - [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); - - /// - public readonly ref TKey GetContextRecordKey(ref AsyncIOContext ctx) => ref ctx.key; - - /// - public readonly ref TValue GetContextRecordValue(ref AsyncIOContext ctx) => ref ctx.value; - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetKeyContainer(ref TKey key) => new StandardHeapContainer(ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetValueContainer(ref TValue value) => new StandardHeapContainer(ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() - => BlittableAllocatorImpl.GetSegmentOffsets(); - - /// - public readonly int OverflowPageCount => _this.OverflowPageCount; - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ref TKey key, long physicalAddress) - => BlittableAllocatorImpl.SerializeKey(ref key, physicalAddress); - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs index 38e72056ae6..69c11b33e73 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableFrame.cs @@ -45,6 +45,14 @@ public long GetPhysicalAddress(long frameNumber, long offset) return pointers[frameNumber % frameSize] + offset; } + public unsafe (byte[] array, long offset) GetArrayAndUnalignedOffset(long frameNumber, long alignedOffset) + { + var frameIndex = frameNumber % frameSize; + + long ptr = (long)Unsafe.AsPointer(ref frame[frameIndex]); + return (frame[frameIndex], alignedOffset + ptr - pointers[frameIndex]); + } + public void Dispose() { } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableScanIterator.cs deleted file mode 100644 index 51c094f22ad..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableScanIterator.cs +++ /dev/null @@ -1,284 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Diagnostics; -using System.Runtime.CompilerServices; -using System.Threading; -using Microsoft.Extensions.Logging; - -namespace Tsavorite.core -{ - /// - /// Scan iterator for hybrid log - /// - public sealed class BlittableScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator - where TStoreFunctions : IStoreFunctions - { - private readonly TsavoriteKV> store; - private readonly BlittableAllocatorImpl hlog; - private readonly BlittableFrame frame; - private readonly bool forceInMemory; - - private TKey currentKey; - private TValue currentValue; - private long framePhysicalAddress; - - /// - /// Constructor for use with head-to-tail scan - /// - /// - /// 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 BlittableScanIterator(TsavoriteKV> store, BlittableAllocatorImpl hlog, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords, LightEpoch epoch, bool forceInMemory = false, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - this.forceInMemory = forceInMemory; - if (frameSize > 0) - frame = new BlittableFrame(frameSize, hlog.PageSize, hlog.GetDeviceSectorSize()); - } - - /// - /// Constructor for use with tail-to-head push iteration of the passed key's record versions - /// - internal BlittableScanIterator(TsavoriteKV> store, BlittableAllocatorImpl hlog, - long beginAddress, LightEpoch epoch, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, hlog.GetTailAddress(), ScanBufferingMode.SinglePageBuffering, false, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - forceInMemory = false; - if (frameSize > 0) - frame = new BlittableFrame(frameSize, hlog.PageSize, hlog.GetDeviceSectorSize()); - } - - /// - /// Get a reference to the current key - /// - public ref TKey GetKey() => ref framePhysicalAddress != 0 ? ref hlog._wrapper.GetKey(framePhysicalAddress) : ref currentKey; - - /// - /// Get a reference to the current value - /// - public ref TValue GetValue() => ref framePhysicalAddress != 0 ? ref hlog._wrapper.GetValue(framePhysicalAddress) : ref currentValue; - - /// - public bool SnapCursorToLogicalAddress(ref long cursor) - { - Debug.Assert(currentAddress == -1, "SnapCursorToLogicalAddress must be called before GetNext()"); - beginAddress = nextAddress = hlog.SnapToFixedLengthLogicalAddressBoundary(ref cursor, BlittableAllocatorImpl.RecordSize); - return true; - } - - /// - /// Get next record in iterator - /// - /// True if record found, false if end of scan - public unsafe bool GetNext(out RecordInfo recordInfo) - { - recordInfo = default; - - while (true) - { - currentAddress = nextAddress; - var stopAddress = endAddress < hlog.GetTailAddress() ? endAddress : hlog.GetTailAddress(); - if (currentAddress >= stopAddress) - return false; - - epoch?.Resume(); - var headAddress = hlog.HeadAddress; - - if (currentAddress < hlog.BeginAddress && !forceInMemory) - currentAddress = hlog.BeginAddress; - - // If currentAddress < headAddress and we're not buffering and not guaranteeing the records are in memory, fail. - if (frameSize == 0 && currentAddress < headAddress && !forceInMemory) - { - epoch?.Suspend(); - throw new TsavoriteException("Iterator address is less than log HeadAddress in memory-scan mode"); - } - - var currentPage = currentAddress >> hlog.LogPageSizeBits; - var offset = currentAddress & hlog.PageSizeMask; - - if (currentAddress < headAddress && !forceInMemory) - BufferAndLoad(currentAddress, currentPage, currentPage % frameSize, headAddress, stopAddress); - - long physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); - var recordSize = hlog._wrapper.GetRecordSize(physicalAddress).Item2; - - // If record does not fit on page, skip to the next page. - if ((currentAddress & hlog.PageSizeMask) + recordSize > hlog.PageSize) - { - nextAddress = (1 + (currentAddress >> hlog.LogPageSizeBits)) << hlog.LogPageSizeBits; - epoch?.Suspend(); - continue; - } - - nextAddress = currentAddress + recordSize; - - recordInfo = hlog._wrapper.GetInfo(physicalAddress); - bool skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (skipOnScan || recordInfo.IsNull()) - { - epoch?.Suspend(); - continue; - } - - OperationStackContext> stackCtx = default; - try - { - // Lock to ensure no value tearing while copying to temp storage. We cannot use GetKey() because it has not yet been set. - if (currentAddress >= headAddress && store is not null) - store.LockForScan(ref stackCtx, ref hlog._wrapper.GetKey(physicalAddress)); - _ = CopyDataMembers(physicalAddress); - } - finally - { - if (stackCtx.recSrc.HasLock) - store.UnlockForScan(ref stackCtx); - } - - // 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(ref TKey key, out RecordInfo recordInfo, out bool continueOnDisk) - { - recordInfo = default; - continueOnDisk = false; - - while (true) - { - // "nextAddress" is reused as "previous address" for this operation. - currentAddress = nextAddress; - if (currentAddress < hlog.HeadAddress) - { - continueOnDisk = currentAddress >= hlog.BeginAddress; - return false; - } - - epoch?.Resume(); - var headAddress = hlog.HeadAddress; - - var currentPage = currentAddress >> hlog.LogPageSizeBits; - var offset = currentAddress & hlog.PageSizeMask; - - long physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); - - recordInfo = hlog._wrapper.GetInfo(physicalAddress); - nextAddress = recordInfo.PreviousAddress; - - // Do not SkipOnScan here; we Seal previous versions. - if (recordInfo.IsNull() || !hlog._storeFunctions.KeysEqual(ref hlog._wrapper.GetKey(physicalAddress), ref key)) - { - epoch?.Suspend(); - continue; - } - - // Success; defer epoch?.Suspend(); to EndGetPrevInMemory - return CopyDataMembers(physicalAddress); - } - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IPushScanIterator.EndGetPrevInMemory() - { - epoch?.Suspend(); - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - long GetPhysicalAddress(long currentAddress, long headAddress, long currentPage, long offset) - { - if (currentAddress >= headAddress) - { - // physicalAddress is in memory; set framePhysicalAddress to 0 so we'll set currentKey and currentValue from physicalAddress below - framePhysicalAddress = 0; - return hlog.GetPhysicalAddress(currentAddress); - } - - // physicalAddress is not in memory, so we'll GetKey and GetValue will use framePhysicalAddress - return framePhysicalAddress = frame.GetPhysicalAddress(currentPage % frameSize, offset); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool CopyDataMembers(long physicalAddress) - { - if (framePhysicalAddress == 0) - { - // Copy the values from the log to data members so we have no ref into the log after the epoch.Suspend(). - currentKey = hlog._wrapper.GetKey(physicalAddress); - currentValue = hlog._wrapper.GetValue(physicalAddress); - } - return true; - } - - /// - /// Get next record in iterator - /// - public bool GetNext(out RecordInfo recordInfo, out TKey key, out TValue value) - { - if (GetNext(out recordInfo)) - { - key = GetKey(); - value = GetValue(); - return true; - } - - key = default; - value = default; - return false; - } - - /// - /// Dispose iterator - /// - public override void Dispose() - { - base.Dispose(); - frame?.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) - => hlog.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice); - - private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) - { - var result = (PageAsyncReadResult)context; - - if (errorCode != 0) - { - logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); - result.cts?.Cancel(); - } - - if (result.freeBuffer1 != null) - { - BlittableAllocatorImpl.PopulatePage(result.freeBuffer1.GetValidPointer(), result.freeBuffer1.required_bytes, result.page); - result.freeBuffer1.Return(); - result.freeBuffer1 = null; - } - - if (errorCode == 0) - result.handle?.Signal(); - - Interlocked.MemoryBarrier(); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/CountdownWrapper.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/CountdownWrapper.cs new file mode 100644 index 00000000000..413b7b3d9e7 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/CountdownWrapper.cs @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +#pragma warning disable 0162 + +using System.Diagnostics; +using System.Threading; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + internal sealed class CountdownWrapper + { + // Separate event for sync code and tcs for async code: Do not block on async code. + private readonly CountdownEvent syncEvent; + private readonly TaskCompletionSource asyncTcs; + int remaining; + + internal CountdownWrapper(int count, bool isAsync) + { + if (isAsync) + { + asyncTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + remaining = count; + return; + } + syncEvent = new CountdownEvent(count); + } + + internal bool IsCompleted => syncEvent is null ? remaining == 0 : syncEvent.IsSet; + + internal void Wait() => syncEvent.Wait(); + internal async ValueTask WaitAsync(CancellationToken cancellationToken) + { + using var reg = cancellationToken.Register(() => asyncTcs.TrySetCanceled()); + await asyncTcs.Task.ConfigureAwait(false); + } + + internal void Decrement() + { + if (asyncTcs is not null) + { + Debug.Assert(remaining > 0); + if (Interlocked.Decrement(ref remaining) == 0) + asyncTcs.TrySetResult(0); + return; + } + syncEvent.Signal(); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs new file mode 100644 index 00000000000..766b0445ddc --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/DiskLogRecord.cs @@ -0,0 +1,840 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Buffers; +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). + /// + 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 static int InitialIOSize => 4 * 1024; + + /// The physicalAddress in the log. + internal long physicalAddress; + + /// The deserialized ValueObject if this is a disk record for the Object Store. Held directly; does not use . + internal IHeapObject valueObject; + + /// 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"); + } + + /// 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(); + } + + /// 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 that takes an from which it obtains the physical address and ValueObject. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal DiskLogRecord(ref AsyncIOContext ctx) : this(ctx.record) + { + valueObject = ctx.ValueObject; + } + + /// 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() + => 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) + { + 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; + } + + /// 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); + } + } + + internal readonly (long length, long dataAddress) ValueInfo + { + [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 + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static byte CreateIndicatorByte(int keyLength, long valueLength, out int keyByteCount, out int valueByteCount) + { + 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 + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int GetByteCount(long num) + { + var result = 0; + do + { + num >>= 8; + ++result; + } while (num > 0); + return result; + } + + static void WriteVarBytes(long value, int len, ref byte* ptr) + { + for (; len > 0; --len) + { + *ptr++ = (byte)(value & 0xFF); + value >>= 8; + } + Debug.Assert(value == 0, "len too short"); + } + + static long ReadVarBytes(int len, ref byte* ptr) + { + long value = 0; + for (var ii = 0; ii < len; ++ii) + value |= (long)*ptr++ << (ii * 8); + return value; + } + + public void Dispose() + { + recordBuffer?.Dispose(); + recordBuffer = null; + } + + #region ISourceLogRecord + /// + public readonly bool IsPinnedKey => true; + + /// + public readonly byte* PinnedKeyPointer + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return IsPinnedKey ? (byte*)KeyInfo.dataAddress : null; } + } + + /// + public readonly bool IsPinnedValue => !Info.ValueIsObject; // We store all bytes inline, but we don't set ValueIsInline, per discussion in SerializeCommonVarByteFields. + + /// + public readonly byte* PinnedValuePointer + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return IsPinnedValue ? (byte*)ValueInfo.dataAddress : null; } + } + + /// + public readonly bool IsSet => physicalAddress != 0; + /// + public readonly ref RecordInfo InfoRef => ref Unsafe.AsRef((byte*)physicalAddress); + /// + public readonly RecordInfo Info => *(RecordInfo*)physicalAddress; + /// + public readonly ReadOnlySpan Key + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + var (length, dataAddress) = KeyInfo; + return new((byte*)dataAddress, length); + } + } + + /// + 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 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 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 long ETag => Info.HasETag ? *(long*)GetETagAddress() : LogRecord.NoETag; + + /// + public readonly long Expiration => Info.HasExpiration ? *(long*)GetExpirationAddress() : 0; + + /// + public readonly void ClearValueObject(Action disposer) { } // Nothing done here; we dispose the object in the pending operation completion + + /// + public readonly bool AsLogRecord(out LogRecord logRecord) + { + logRecord = default; + return false; + } + + /// + public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) + { + diskLogRecord = this; + return true; + } + + /// + [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 int OptionalLength => ETagLen + ExpirationLen; + + private readonly int ETagLen => Info.HasETag ? LogRecord.ETagSize : 0; + private readonly int ExpirationLen => Info.HasExpiration ? LogRecord.ExpirationSize : 0; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private readonly long GetOptionalStartAddress() + { + var (length, dataAddress) = ValueInfo; + return dataAddress + length; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private readonly long GetETagAddress() => GetOptionalStartAddress(); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private readonly long GetExpirationAddress() => GetETagAddress() + ETagLen; + + #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); + + /// + /// Serialize for Read or RMW operations, called by PendingContext; these have no Value but have TInput, TOutput, and TContext, which are handled by PendingContext. + /// + /// 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) + { + // 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; + + allocatedRecord.pool.EnsureSize(ref allocatedRecord, (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(ref readonly 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(ref readonly LogRecord logRecord, SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer, ref SectorAlignedMemory allocatedRecord) + { + if (logRecord.Info.RecordIsInline) + { + DirectCopyRecord(logRecord.ActualRecordSize, logRecord.physicalAddress, bufferPool, ref allocatedRecord); + return; + } + + // 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 + { + if (valueSerializer is not null) + { + var stream = new UnmanagedMemoryStream(ptr, logRecord.ValueObject.DiskSize, logRecord.ValueObject.DiskSize, FileAccess.ReadWrite); + valueSerializer.BeginSerialize(stream); + valueSerializer.Serialize(logRecord.ValueObject); + valueSerializer.EndSerialize(); + } + 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. + /// + /// If ., it points directly to the network buffer so we include the length prefix in the output. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private static long SerializeVarbyteRecord(ref TSourceLogRecord srcLogRecord, long srcPhysicalAddress, IObjectSerializer valueSerializer, + ref SpanByteAndMemory output, MemoryPool memoryPool) + where TSourceLogRecord : ISourceLogRecord + { + // 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 + { + var outPtr = output.SpanByte.ToPointer(); + *(int*)outPtr = (int)recordSize; + serializedSize = SerializeVarbyteRecordToPinnedPointer(ref srcLogRecord, outPtr + sizeof(int), indicatorByte, keyLengthByteCount, valueLength, valueLengthByteCount, valueSerializer); + output.SpanByte.Length = (int)serializedSize; + } + 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); + fixed (byte* outPtr = output.MemorySpan) + { + serializedSize = SerializeVarbyteRecordToPinnedPointer(ref srcLogRecord, outPtr, indicatorByte, keyLengthByteCount, valueLength, valueLengthByteCount, valueSerializer); + output.Length = (int)recordSize; + } + } + Debug.Assert(serializedSize == recordSize, $"Serialized size {serializedSize} does not match expected size {recordSize}"); + return serializedSize; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private static long SerializeVarbyteRecordToPinnedPointer(ref TSourceLogRecord srcLogRecord, byte* ptr, byte indicatorByte, + int keyLengthByteCount, long valueLength, int valueLengthByteCount, IObjectSerializer valueSerializer) + where TSourceLogRecord : ISourceLogRecord + { + var dstRecordInfoPtr = (RecordInfo*)ptr; + ptr = SerializeCommonVarByteFields(srcLogRecord.Info, indicatorByte, srcLogRecord.Key, keyLengthByteCount, valueLength, valueLengthByteCount, ptr); + + // 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 + { + var stream = new UnmanagedMemoryStream(ptr, srcLogRecord.ValueObject.DiskSize, srcLogRecord.ValueObject.DiskSize, FileAccess.ReadWrite); + valueSerializer.BeginSerialize(stream); + valueSerializer.Serialize(srcLogRecord.ValueObject); + valueSerializer.EndSerialize(); + } + ptr += valueLength; + + if (srcLogRecord.Info.HasETag) + { + dstRecordInfoPtr->SetHasETag(); + *(long*)ptr = srcLogRecord.ETag; + ptr += LogRecord.ETagSize; + } + + if (srcLogRecord.Info.HasExpiration) + { + dstRecordInfoPtr->SetHasExpiration(); + *(long*)ptr = srcLogRecord.Expiration; + ptr += LogRecord.ExpirationSize; + } + + 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); + 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; + } + + if (logRecord.Info.HasExpiration) + { + InfoRef.SetHasExpiration(); + *(long*)ptr = logRecord.Expiration; + ptr += LogRecord.ExpirationSize; + } + } + + /// + /// 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. + /// + /// + /// 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) + { + Debug.Assert(inputDiskLogRecord.ValueInfo.length > 0, "inputDiskLogRecord value length should be > 0"); + DirectCopyRecord(inputDiskLogRecord.GetSerializedLength(), inputDiskLogRecord.physicalAddress, bufferPool, ref allocatedRecord); + return; + } + + // 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(ref TSourceLogRecord srcLogRecord, IObjectSerializer valueSerializer, ref SpanByteAndMemory output, MemoryPool memoryPool) + where TSourceLogRecord : ISourceLogRecord + { + if (srcLogRecord.AsLogRecord(out var logRecord)) + { + if (logRecord.Info.RecordIsInline) + DirectCopyRecord(logRecord.physicalAddress, logRecord.ActualRecordSize, ref output, memoryPool); + else + _ = SerializeVarbyteRecord(ref logRecord, logRecord.physicalAddress, valueSerializer, ref output, memoryPool); + return; + } + + 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))) + { + DirectCopyRecord(diskLogRecord.physicalAddress, diskLogRecord.GetSerializedLength(), ref output, memoryPool); + return; + } + _ = SerializeVarbyteRecord(ref diskLogRecord, diskLogRecord.physicalAddress, valueSerializer, ref output, memoryPool); + } + #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}"; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocator.cs deleted file mode 100644 index bc14667d390..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocator.cs +++ /dev/null @@ -1,176 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Runtime.CompilerServices; - -namespace Tsavorite.core -{ - /// - /// Struct wrapper (for inlining) around the fixed-length Blittable allocator. - /// - public struct GenericAllocator : IAllocator - where TStoreFunctions : IStoreFunctions - { - /// The wrapped class containing all data and most actual functionality. This must be the ONLY field in this structure so its size is sizeof(IntPtr). - private readonly GenericAllocatorImpl _this; - - public GenericAllocator(AllocatorSettings settings, TStoreFunctions storeFunctions) - { - // Called by TsavoriteKV via allocatorCreator; must pass a wrapperCreator to AllocatorBase - _this = new(settings, storeFunctions, @this => new GenericAllocator(@this)); - } - - public GenericAllocator(object @this) - { - // Called by AllocatorBase via primary ctor wrapperCreator - _this = (GenericAllocatorImpl)@this; - } - - /// - 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.GetStartLogicalAddress(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddress(page); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref RecordInfo GetInfo(long physicalAddress) => ref _this.GetInfo(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe ref RecordInfo GetInfoFromBytePointer(byte* ptr) => ref _this.GetInfoFromBytePointer(ptr); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TKey GetKey(long physicalAddress) => ref _this.GetKey(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TValue GetValue(long physicalAddress) => ref _this.GetValue(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref TValue GetAndInitializeValue(long physicalAddress, long endPhysicalAddress) => ref GetValue(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) => _this.GetRecordSize(physicalAddress); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref TKey key, ref TInput input, ref TValue value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - where TVariableLengthInput : IVariableLengthInput - => _this.GetRMWCopyDestinationRecordSize(ref key, ref input, ref value, ref recordInfo, varlenInput); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref TKey key) - => _this.GetTombstoneRecordSize(ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetRequiredRecordSize(long physicalAddress, int availableBytes) => GetAverageRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetAverageRecordSize() => _this.GetAverageRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetFixedRecordSize() => _this.GetFixedRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref TKey key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => _this.GetRMWInitialRecordSize(ref key, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref TKey key, ref TValue value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => _this.GetUpsertRecordSize(ref key, ref value, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref TKey key, ref TValue value) => _this.GetRecordSize(ref key, ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetValueLength(ref TValue value) => _this.GetValueLength(ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) => _this.RetrievedFullRecord(record, ref ctx); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void AllocatePage(int pageIndex) => _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); - - /// - [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); - - /// - public readonly ref TKey GetContextRecordKey(ref AsyncIOContext ctx) => ref ctx.key; - - /// - public readonly ref TValue GetContextRecordValue(ref AsyncIOContext ctx) => ref ctx.value; - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetKeyContainer(ref TKey key) => new StandardHeapContainer(ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetValueContainer(ref TValue value) => new StandardHeapContainer(ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() => _this.GetSegmentOffsets(); - - /// - public readonly int OverflowPageCount => _this.OverflowPageCount; - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ref TKey key, long physicalAddress) => _this.SerializeKey(ref key, physicalAddress); - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericFrame.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/GenericFrame.cs deleted file mode 100644 index cb3e78abef4..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericFrame.cs +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Runtime.CompilerServices; - -namespace Tsavorite.core -{ - /// - /// A frame is an in-memory circular buffer of log pages - /// - internal sealed class GenericFrame : IDisposable - { - private readonly AllocatorRecord[][] frame; - public readonly int frameSize, pageSize; - private static int RecordSize => Unsafe.SizeOf>(); - - public GenericFrame(int frameSize, int pageSize) - { - this.frameSize = frameSize; - this.pageSize = pageSize; - frame = new AllocatorRecord[frameSize][]; - } - - public void Allocate(int index) - { - frame[index] = new AllocatorRecord[(pageSize + RecordSize - 1) / RecordSize]; - } - - public void Clear(int pageIndex) - { - Array.Clear(frame[pageIndex], 0, frame[pageIndex].Length); - } - - public ref TKey GetKey(long frameNumber, long offset) - { - return ref frame[frameNumber][offset].key; - } - - public ref TValue GetValue(long frameNumber, long offset) - { - return ref frame[frameNumber][offset].value; - } - - public ref RecordInfo GetInfo(long frameNumber, long offset) - { - return ref frame[frameNumber][offset].info; - } - - public ref AllocatorRecord[] GetPage(long frameNumber) - { - return ref frame[frameNumber]; - } - - public void Dispose() - { - Array.Clear(frame, 0, frame.Length); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/GenericScanIterator.cs deleted file mode 100644 index f72eb099e4f..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericScanIterator.cs +++ /dev/null @@ -1,294 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Diagnostics; -using System.Threading; -using Microsoft.Extensions.Logging; - -namespace Tsavorite.core -{ - /// - /// Scan iterator for hybrid log - /// - internal sealed class GenericScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator - where TStoreFunctions : IStoreFunctions - { - private readonly TsavoriteKV> store; - private readonly GenericAllocatorImpl hlog; - private readonly GenericFrame frame; - private readonly int recordSize; - - private TKey currentKey; - private TValue currentValue; - - private long currentPage = -1, currentOffset = -1, currentFrame = -1; - - /// - /// Constructor - /// - public GenericScanIterator(TsavoriteKV> store, GenericAllocatorImpl hlog, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords, LightEpoch epoch, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - recordSize = hlog.GetRecordSize(0).allocatedSize; - if (frameSize > 0) - frame = new GenericFrame(frameSize, hlog.PageSize); - } - - /// - /// Constructor for use with tail-to-head push iteration of the passed key's record versions - /// - public GenericScanIterator(TsavoriteKV> store, GenericAllocatorImpl hlog, - long beginAddress, LightEpoch epoch, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, hlog.GetTailAddress(), ScanBufferingMode.SinglePageBuffering, false, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - recordSize = hlog.GetRecordSize(0).allocatedSize; - if (frameSize > 0) - frame = new GenericFrame(frameSize, hlog.PageSize); - } - - /// - /// Gets reference to current key - /// - /// - public ref TKey GetKey() => ref currentKey; - - /// - /// Gets reference to current value - /// - /// - public ref TValue GetValue() => ref currentValue; - - /// - public bool SnapCursorToLogicalAddress(ref long cursor) - { - Debug.Assert(currentAddress == -1, "SnapCursorToLogicalAddress must be called before GetNext()"); - beginAddress = nextAddress = hlog.SnapToFixedLengthLogicalAddressBoundary(ref cursor, GenericAllocatorImpl.RecordSize); - return true; - } - - /// - /// Get next record in iterator - /// - /// True if record found, false if end of scan - public unsafe bool GetNext(out RecordInfo recordInfo) - { - recordInfo = default; - currentKey = default; - currentValue = default; - currentPage = currentOffset = currentFrame = -1; - - while (true) - { - currentAddress = nextAddress; - var stopAddress = endAddress < hlog.GetTailAddress() ? endAddress : hlog.GetTailAddress(); - if (currentAddress >= stopAddress) - return false; - - epoch?.Resume(); - try - { - var headAddress = hlog.HeadAddress; - - if (currentAddress < hlog.BeginAddress) - currentAddress = hlog.BeginAddress; - - // If currentAddress < headAddress and we're not buffering, fail. - if (frameSize == 0 && currentAddress < headAddress) - { - throw new TsavoriteException("Iterator address is less than log HeadAddress in memory-scan mode"); - } - - currentPage = currentAddress >> hlog.LogPageSizeBits; - currentOffset = (currentAddress & hlog.PageSizeMask) / recordSize; - - if (currentAddress < headAddress) - _ = BufferAndLoad(currentAddress, currentPage, currentPage % frameSize, headAddress, stopAddress); - - // Check if record fits on page, if not skip to next page - if ((currentAddress & hlog.PageSizeMask) + recordSize > hlog.PageSize) - { - nextAddress = (1 + (currentAddress >> hlog.LogPageSizeBits)) << hlog.LogPageSizeBits; - continue; - } - - nextAddress = currentAddress + recordSize; - - if (currentAddress >= headAddress) - { - // Read record from cached page memory - currentPage %= hlog.BufferSize; - currentFrame = -1; // Frame is not used in this case. - - recordInfo = hlog.values[currentPage][currentOffset].info; - bool _skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (_skipOnScan) - { - continue; - } - - // Copy the object values from cached page memory to data members; we have no ref into the log after the epoch.Suspend(). - // These are pointer-sized shallow copies but we need to lock to ensure no value tearing inside the object while copying to temp storage. - OperationStackContext> stackCtx = default; - try - { - // We cannot use GetKey() because it has not yet been set. - if (currentAddress >= headAddress && store is not null) - store.LockForScan(ref stackCtx, ref hlog.values[currentPage][currentOffset].key); - - recordInfo = hlog.values[currentPage][currentOffset].info; - currentKey = hlog.values[currentPage][currentOffset].key; - currentValue = hlog.values[currentPage][currentOffset].value; - } - finally - { - if (stackCtx.recSrc.HasLock) - store.UnlockForScan(ref stackCtx); - } - - // Success - return true; - } - - currentFrame = currentPage % frameSize; - recordInfo = frame.GetInfo(currentFrame, currentOffset); - bool skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (skipOnScan || recordInfo.IsNull()) - { - continue; - } - - // Copy the object values from the frame to data members. - currentKey = frame.GetKey(currentFrame, currentOffset); - currentValue = frame.GetValue(currentFrame, currentOffset); - currentPage = currentOffset = -1; - } - 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(ref TKey key, out RecordInfo recordInfo, out bool continueOnDisk) - { - recordInfo = default; - currentKey = default; - currentValue = default; - currentPage = currentOffset = currentFrame = -1; - continueOnDisk = false; - - while (true) - { - // "nextAddress" is reused as "previous address" for this operation. - currentAddress = nextAddress; - if (currentAddress < hlog.HeadAddress) - { - continueOnDisk = currentAddress >= hlog.BeginAddress; - return false; - } - - epoch?.Resume(); - - currentPage = currentAddress >> hlog.LogPageSizeBits; - currentOffset = (currentAddress & hlog.PageSizeMask) / recordSize; - - // Read record from cached page memory - currentPage %= hlog.BufferSize; - currentFrame = -1; // Frame is not used in this case. - - recordInfo = hlog.values[currentPage][currentOffset].info; - nextAddress = currentAddress + recordSize; - - bool skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (skipOnScan || recordInfo.IsNull() || !hlog._storeFunctions.KeysEqual(ref hlog.values[currentPage][currentOffset].key, ref key)) - { - epoch?.Suspend(); - continue; - } - - // Copy the object values from cached page memory to data members; we have no ref into the log after the epoch.Suspend(). - // These are pointer-sized shallow copies. - recordInfo = hlog.values[currentPage][currentOffset].info; - currentKey = hlog.values[currentPage][currentOffset].key; - currentValue = hlog.values[currentPage][currentOffset].value; - - // Success; defer epoch?.Suspend(); to EndGet - return true; - } - } - - bool IPushScanIterator.EndGetPrevInMemory() - { - epoch?.Suspend(); - return true; - } - - /// - /// Get next record using iterator - /// - /// - /// - /// - /// - public bool GetNext(out RecordInfo recordInfo, out TKey key, out TValue value) - { - if (GetNext(out recordInfo)) - { - key = currentKey; - value = currentValue; - return true; - } - - key = default; - value = default; - return false; - } - - /// - /// Dispose iterator - /// - public override void Dispose() - { - base.Dispose(); - frame?.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) - => hlog.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice); - - private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) - { - var result = (PageAsyncReadResult)context; - - if (errorCode != 0) - { - logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); - result.cts?.Cancel(); - } - - if (result.freeBuffer1 != null) - { - hlog.PopulatePage(result.freeBuffer1.GetValidPointer(), result.freeBuffer1.required_bytes, ref frame.GetPage(result.page % frame.frameSize)); - result.freeBuffer1.Return(); - result.freeBuffer1 = null; - } - - if (errorCode == 0) - _ = result.handle?.Signal(); - - Interlocked.MemoryBarrier(); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs index 967e115ac6c..d20a07a7085 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocator.cs @@ -1,48 +1,54 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// /// Interface for hybrid log memory allocator struct wrapper for inlining. This contains the performance-critical methods that must be inlined; - /// abstract/virtual methods may be called via . + /// abstract/virtual methods may be called via . /// - public interface IAllocator : IAllocatorCallbacks - where TStoreFunctions : IStoreFunctions + public interface IAllocator : IAllocatorCallbacks + where TStoreFunctions : IStoreFunctions { /// The base class instance of the allocator implementation - AllocatorBase GetBase() - where TAllocator : IAllocator; - - /// Whether this allocator uses fixed-length records - bool IsFixedLength { get; } - - /// Whether this allocator uses a separate object log - bool HasObjectLog { get; } + AllocatorBase GetBase() + where TAllocator : IAllocator; - /// Cast address range to . For this will also initialize the value to span the address range. - ref TValue GetAndInitializeValue(long physicalAddress, long endPhysicalAddress); + /// Initialize the value to span the address range. + /// The start of the record (address of its ). + /// The record size info, which tells us the value size and whether that is overflow. + void InitializeValue(long physicalAddress, ref RecordSizeInfo sizeInfo); /// Get copy destination size for RMW, taking Input into account - (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref TKey key, ref TInput input, ref TValue value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - where TVariableLengthInput : IVariableLengthInput; + RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput; /// Get initial record size for RMW, given the and - (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref TKey key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput; + RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput; + + /// Get record size required for the given , , and + RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput; /// Get record size required for the given , , and - (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref TKey key, ref TValue value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput; + RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput; - /// Get record size required for the given and - (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref TKey key, ref TValue value); + /// Get record size required for the given , , and + RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput; - /// Get the record size for a tombstoned record - (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref TKey key); + /// Get record size required for a new tombstone record + RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key); - /// Get the size of the given - int GetValueLength(ref TValue value); + /// Get record size required to allocate a new record. Includes allocator-specific information such as key and value overflow. + /// Requires to be populated already. + void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo); /// Mark the page that contains as dirty void MarkPage(long logicalAddress, long version); @@ -51,9 +57,21 @@ AllocatorBase GetBase() void MarkPageAtomic(long logicalAddress, long version); /// Get segment offsets - long[] GetSegmentOffsets(); + long[] GetSegmentOffsets(); // TODO remove /// Serialize key to log - void SerializeKey(ref TKey key, long physicalAddress); + 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); + + /// Dispose an in-memory log record + void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason); + + /// Dispose an on-disk log record + void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs index 6c74dea3dcc..28f44c23be9 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IAllocatorCallbacks.cs @@ -5,12 +5,12 @@ 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. /// /// This interface does not currently appear in type constraints, but the organization may prove useful. - public interface IAllocatorCallbacks - where TStoreFunctions : IStoreFunctions + public interface IAllocatorCallbacks + where TStoreFunctions : IStoreFunctions { /// Get start logical address on long GetStartLogicalAddress(long page); @@ -21,27 +21,6 @@ public interface IAllocatorCallbacks /// Get physical address from long GetPhysicalAddress(long logicalAddress); - /// Get from - ref RecordInfo GetInfo(long physicalAddress); - - /// Get from pinned memory - unsafe ref RecordInfo GetInfoFromBytePointer(byte* ptr); - - /// Get from - ref TKey GetKey(long physicalAddress); - - /// Get from - ref TValue GetValue(long physicalAddress); - - /// Get the actual (used) and allocated record sizes at - (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress); - - /// Get number of bytes required to read the full record that starts at for . - int GetRequiredRecordSize(long physicalAddress, int availableBytes); - - /// Get average record size - int GetAverageRecordSize(); - /// Allocate the page in the circular buffer slot at void AllocatePage(int pageIndex); @@ -61,22 +40,5 @@ public interface IAllocatorCallbacks /// Number of extra overflow pages allocated int OverflowPageCount { get; } - - int GetFixedRecordSize(); - - /// Retrieve key from IO context record - ref TKey GetContextRecordKey(ref AsyncIOContext ctx); - - /// Retrieve value from IO context record - ref TValue GetContextRecordValue(ref AsyncIOContext ctx); - - /// Determine whether we IO has returned the full record - unsafe bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx); - - /// Get heap container for pending key - IHeapContainer GetKeyContainer(ref TKey key); - - /// Get heap container for pending value - IHeapContainer GetValueContainer(ref TValue value); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs new file mode 100644 index 00000000000..459e80e8db4 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IHeapObject.cs @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Tsavorite.core +{ + /// + /// This is the base interface from which any value of object type must derive for . + /// + public interface IHeapObject : IDisposable + { + /// + /// Total size of the object in memory, including .NET object overheads. + /// + long MemorySize { get; set; } + + /// + /// Total serialized size of the object; the size it will take when written to disk or other storage. + /// + long DiskSize { get; set; } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IScanIteratorFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IScanIteratorFunctions.cs index e163c6b72dd..6d79b2e129c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IScanIteratorFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IScanIteratorFunctions.cs @@ -35,7 +35,7 @@ public enum CursorRecordResult /// /// Callback functions for log scan or key-version iteration /// - public interface IScanIteratorFunctions + public interface IScanIteratorFunctions { /// Iteration is starting. /// Start address of the scan @@ -43,41 +43,31 @@ public interface IScanIteratorFunctions /// True to continue iteration, else false bool OnStart(long beginAddress, long endAddress); - /// Next record in iteration for a record not in mutable log memory. - /// Reference to the current record's key - /// Reference to the current record's Value + /// Read the next record in the iteration. + /// Reference to the current log record's info /// Record metadata, including and the current record's logical address /// The number of records accepted so far, not including the current one. /// Indicates whether the current record was accepted, or whether to end the current ScanCursor call. /// Ignored for non-cursor Scans; set to . /// True to continue iteration, else false - bool SingleReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult); - - /// Next record in iteration for a record in mutable log memory. - /// Reference to the current record's key - /// Reference to the current record's Value - /// Record metadata, including and the current record's logical address - /// The number of records accepted so far, not including the current one. - /// Indicates whether the current record was accepted, or whether to end the current ScanCursor call. - /// Ignored for non-cursor Scans; set to . - /// True to continue iteration, else false - bool ConcurrentReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult); + bool Reader(ref TSourceLogRecord logRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord; /// Iteration is complete. /// If true, the iteration completed; else scanFunctions.*Reader() returned false to stop the iteration. /// The number of records returned before the iteration stopped. void OnStop(bool completed, long numberOfRecords); - /// An exception was thrown on iteration (likely during or . + /// An exception was thrown on iteration (likely during . /// The exception that was thrown. /// The number of records returned, including the current one, before the exception. void OnException(Exception exception, long numberOfRecords); } - internal interface IPushScanIterator + internal interface IPushScanIterator { - bool BeginGetPrevInMemory(ref TKey key, out RecordInfo recordInfo, out bool continueOnDisk); - bool EndGetPrevInMemory(); + bool BeginGetPrevInMemory(ReadOnlySpan key, out LogRecord logRecord, out bool continueOnDisk); + void EndGetPrevInMemory(); /// /// When beginning a cursor scan, if it is not the last cursor returned, snap it to the preceding logical address boundary. diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs new file mode 100644 index 00000000000..ba81979b366 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ISourceLogRecord.cs @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Tsavorite.core +{ + /// An interface to cover either an in-memory or on-disk log record for RCU + public unsafe interface ISourceLogRecord + { + /// A ref to the record header + ref RecordInfo InfoRef { get; } + + /// Fast access returning a copy of the record header + RecordInfo Info { get; } + + /// Whether there is actually a record here + bool IsSet { get; } + + /// The key, which may be inline in this record or an overflow byte[] + /// Not a ref return as it cannot be changed + ReadOnlySpan Key { get; } + + /// Whether the record's key is pinned in memory, e.g. inline in the log vs an overflow byte[]. If this is true, is non-null. + bool IsPinnedKey { get; } + + /// The pointer to the pinned memory if is true, else null. + byte* PinnedKeyPointer { get; } + + /// 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 LogRecord.TrySetValueSpan(Span{_byte_}, ref RecordSizeInfo) 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; } + + /// 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). + /// The disposer is not inlined, but this is called after object cloning, so the perf hit won't matter + void ClearValueObject(Action disposer); + + /// 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); + + /// 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); + + /// Get the record's field info, for use in calculating required record size + RecordFieldInfo GetRecordFieldInfo(); + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs index 422353aa745..ebeb5f2cfe6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs @@ -8,7 +8,7 @@ namespace Tsavorite.core /// /// Callback functions for streaming snapshot iteration /// - public interface IStreamingSnapshotIteratorFunctions + public interface IStreamingSnapshotIteratorFunctions { /// Iteration is starting. /// Checkpoint token @@ -18,12 +18,11 @@ public interface IStreamingSnapshotIteratorFunctions bool OnStart(Guid checkpointToken, long currentVersion, long nextVersion); /// Next record in the streaming snapshot. - /// Reference to the current record's key - /// Reference to the current record's Value + /// Reference to the current record /// Record metadata, including and the current record's logical address /// The number of records returned so far, not including the current one. /// True to continue iteration, else false - bool Reader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords); + bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords) where TSourceLogRecord : ISourceLogRecord; /// Iteration is complete. /// If true, the iteration completed; else OnStart() or Reader() returned false to stop the iteration. diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ITsavoriteScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ITsavoriteScanIterator.cs index 3ecb9286e3a..7add4ffb8ea 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ITsavoriteScanIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ITsavoriteScanIterator.cs @@ -6,9 +6,9 @@ namespace Tsavorite.core { /// - /// Scan buffering mode + /// Scan buffering mode when reading from disk /// - public enum ScanBufferingMode + public enum DiskScanBufferingMode { /// /// Buffer only current page being scanned @@ -27,39 +27,31 @@ public enum ScanBufferingMode } /// - /// Scan iterator interface for Tsavorite log + /// Scan buffering mode for in-memory records, e.g. for copying and holding a record for Pull iterators /// - /// - /// - public interface ITsavoriteScanIterator : IDisposable + public enum InMemoryScanBufferingMode { /// - /// Gets reference to current key - /// - /// - ref TKey GetKey(); - - /// - /// Gets reference to current value + /// Buffer the current record being scanned. Automatic for Pull iteration. /// - /// - ref TValue GetValue(); + CurrentRecordBuffering, /// - /// Get next record + /// Do not buffer - with this mode, Push iteration will hold the epoch during each record's push to the client /// - /// - /// True if record found, false if end of scan - bool GetNext(out RecordInfo recordInfo); + NoBuffering + } + /// + /// Scan iterator interface for Tsavorite log + /// + public interface ITsavoriteScanIterator : ISourceLogRecord, IDisposable + { /// /// Get next record /// - /// - /// - /// /// True if record found, false if end of scan - bool GetNext(out RecordInfo recordInfo, out TKey key, out TValue value); + bool GetNext(); /// /// Current address diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs new file mode 100644 index 00000000000..15704884976 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/LogField.cs @@ -0,0 +1,361 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +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 + /// + /// + /// Considerations regarding variable field sizes: + /// + /// Keys are immutable (unless revivification is happening), so the inline size of a Key field does not change + /// When Values change size the Filler length and offsets to optional ETag and Extension are adjusted. Converting between inline and out-of-line + /// due to size changes altering whether the Value overflows is handled as part of normal Value-sizechange operations + /// + /// + public 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) + { + var byteArrayObj = objectIdMap.Get(objectId); + return new Span(Unsafe.As(ref byteArrayObj)); + } + 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. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static Span ConvertInlineToOverflow(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + { + // First copy the data + var array = GC.AllocateUninitializedArray(newLength); + var oldLength = GetInlineLengthRef(fieldAddress); + var copyLength = oldLength < newLength ? oldLength : newLength; + + if (copyLength > 0) + { + var oldSpan = new ReadOnlySpan((byte*)GetInlineDataAddress(fieldAddress), copyLength); + oldSpan.CopyTo(array); + } + + // 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; + } + + /// + /// 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. Here we do not copy the data; we assume the caller will have already + /// 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) + { + var array = GC.AllocateUninitializedArray(newLength); + + var objectId = GetObjectIdRef(fieldAddress); + if (objectId == ObjectIdMap.InvalidObjectId) + { + objectId = objectIdMap.Allocate(); + GetObjectIdRef(fieldAddress) = objectId; + } + objectIdMap.Set(objectId, array); + + recordInfo.SetValueIsOverflow(); + return array; + } + + /// + /// Convert a Span field from inline to ObjectId. + /// + /// + /// Applies to Value 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 a valid ObjectId. 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. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int ConvertInlineToHeapObject(ref RecordInfo recordInfo, long fieldAddress, 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. + 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; + return objectId; + } + + /// + /// Convert a Span field from an out-of-line overflow allocation to ObjectId. + /// + /// + /// Applies to Value 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 a valid ObjectId. 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. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static int ConvertOverflowToHeapObject(ref RecordInfo recordInfo, long fieldAddress, ObjectIdMap objectIdMap) + { + var objectId = GetObjectIdRef(fieldAddress); + if (objectId != ObjectIdMap.InvalidObjectId) + { + // Clear the byte[] from the existing slot + objectIdMap.Set(objectId, null); + } + else + { + objectId = objectIdMap.Allocate(); + GetObjectIdRef(fieldAddress) = objectId; + } + + 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. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static Span ConvertOverflowToInline(ref RecordInfo recordInfo, long fieldAddress, int newLength, ObjectIdMap objectIdMap) + { + // First copy the data + var objectId = GetObjectIdRef(fieldAddress); + if (objectId != ObjectIdMap.InvalidObjectId) + { + var oldSpan = new Span((byte[])objectIdMap.Get(objectId)); + + // 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; + } + return SetInlineDataLength(fieldAddress, newLength); + } + + /// + /// 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) + { + // 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; + + if (objectId != ObjectIdMap.InvalidObjectId) + objectIdMap.Set(objectId, null); + + // Sequencing here is important for zeroinit correctness + GetInlineLengthRef(fieldAddress) = 0; + if (isKey) + recordInfo.SetKeyIsInline(); + else + recordInfo.SetValueIsInline(); + } + + /// + /// Convert a Value field from ObjectId to inline. + /// + /// + /// Applies to Value 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 ObjectId (which may be ObjectIdMap.InvalidObjectId). Here we do not copy the data; we assume + /// 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) + { + ref var objIdRef = ref GetObjectIdRef(fieldAddress); + objectIdMap.Free(objIdRef); + objIdRef = 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); + } + + /// + /// Reallocate a Span field that is overflow, e.g. to make the overflow allocation larger. Shrinkage is done in-place (the caller decides if the + /// 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. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static Span ReallocateOverflow(long fieldAddress, int newLength, ObjectIdMap objectIdMap) + { + byte[] newArray; + + var objectId = GetObjectIdRef(fieldAddress); + if (objectId != ObjectIdMap.InvalidObjectId) + { + var oldArray = (byte[])objectIdMap.Get(objectId); + if (oldArray.Length == newLength) + return new Span(oldArray); + + // Allocate and copy + newArray = new byte[newLength]; + var copyLength = oldArray.Length < newLength ? oldArray.Length : newLength; + Array.Copy(oldArray, newArray, copyLength); + if (copyLength < newLength) + Array.Clear(newArray, copyLength, newLength - copyLength); + } + else + { + // Allocate; nothing to copy + newArray = new byte[newLength]; + objectId = objectIdMap.Allocate(); + GetObjectIdRef(fieldAddress) = objectId; + } + objectIdMap.Set(objectId, newArray); + return new Span(newArray); + } + } +} \ 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 new file mode 100644 index 00000000000..472df971ff7 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/LogRecord.cs @@ -0,0 +1,848 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +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. + /// 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. + internal readonly long physicalAddress; + + /// The ObjectIdMap if this is a record in the object log. + internal readonly ObjectIdMap objectIdMap; + + /// Number of bytes required to store an ETag + public const int ETagSize = sizeof(long); + /// Invalid ETag, and also the pre-incremented value + public const int NoETag = 0; + /// Number of bytes required to store an Expiration + public const int ExpirationSize = sizeof(long); + /// Number of bytes required to store the FillerLen + internal const int FillerLengthSize = sizeof(int); + + /// Address-only ctor. Must only be used for simple record parsing, including inline size calculations. + /// In particular, if knowledge of whether this is a string or object record is required, or an overflow allocator is needed, this method cannot be used. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal LogRecord(long physicalAddress) => this.physicalAddress = physicalAddress; + + /// 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) + : this(physicalAddress) + { + this.objectIdMap = objectIdMap; + } + + #region ISourceLogRecord + /// + public readonly bool IsSet => physicalAddress != 0; + /// + public readonly ref RecordInfo InfoRef + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return ref GetInfoRef(physicalAddress); } + } + + /// + public readonly RecordInfo Info + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return GetInfo(physicalAddress); } + } + + /// + public readonly ReadOnlySpan Key => GetKey(physicalAddress, objectIdMap); + + /// + public bool IsPinnedKey => Info.KeyIsInline; + + /// + public byte* PinnedKeyPointer => IsPinnedKey ? (byte*)LogField.GetInlineDataAddress(KeyAddress) : null; + + /// + public readonly Span ValueSpan + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + Debug.Assert(!Info.ValueIsObject, "ValueSpan is not valid for Object values"); + return LogField.AsSpan(ValueAddress, Info.ValueIsInline, objectIdMap); + } + } + + /// + public readonly IHeapObject ValueObject + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + 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); + } + } + + /// + public readonly ReadOnlySpan RecordSpan + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + if (!Info.RecordIsInline) + throw new TsavoriteException("RecordSpan is not valid for non-inline records"); + return new((byte*)physicalAddress, GetInlineRecordSizes().actualSize); + } + } + + /// + public bool IsPinnedValue => Info.ValueIsInline; + + /// + public byte* PinnedValuePointer => IsPinnedValue ? (byte*)LogField.GetInlineDataAddress(ValueAddress) : null; + + /// + public readonly long ETag => Info.HasETag ? *(long*)GetETagAddress() : NoETag; + /// + public readonly long Expiration => Info.HasExpiration ? *(long*)GetExpirationAddress() : 0; + + /// + [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 + { + Debug.Assert(Info.ValueIsObject, "ClearValueObject() is not valid for Span Values"); + if (Info.ValueIsObject) + { + objectIdMap.ClearAt(ValueObjectId, disposer); + if (!Info.ValueIsInline) + *ValueObjectIdAddress = ObjectIdMap.InvalidObjectId; + } + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly bool AsLogRecord(out LogRecord logRecord) + { + logRecord = this; + return true; + } + + /// + public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) + { + diskLogRecord = default; + return false; + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordFieldInfo GetRecordFieldInfo() => new() + { + KeyDataSize = LogField.GetInlineDataSizeOfField(KeyAddress, Info.KeyIsInline), + ValueDataSize = LogField.GetInlineDataSizeOfField(ValueAddress, Info.ValueIsInline), + ValueIsObject = Info.ValueIsObject, + HasETag = Info.HasETag, + HasExpiration = Info.HasExpiration + }; + #endregion // ISourceLogRecord + + /// + /// The record is directly copyable if it is entirely inline; otherwise it must be Serialized to linear format + /// + public readonly bool IsDirectlyCopyable => Info.RecordIsInline; + + /// A ref to the record header + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ref RecordInfo GetInfoRef(long physicalAddress) => ref *(RecordInfo*)physicalAddress; + + /// Fast access returning a copy of the record header + [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) + { + var keyAddress = GetKeyAddress(physicalAddress); + return keyAddress + LogField.GetInlineTotalSizeOfField(keyAddress, GetInfo(physicalAddress).KeyIsInline); + } + + 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 + { + 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; + } + } + + /// 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 + { + var valueAddress = ValueAddress; + var valueSize = LogField.GetInlineTotalSizeOfField(valueAddress, GetInfo(physicalAddress).ValueIsInline); + return (int)(valueAddress - physicalAddress + valueSize + OptionalLength); + } + } + + /// + /// Asserts that is the same size as the value data size in the before setting the length. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TrySetValueLength(int newValueSize, ref RecordSizeInfo sizeInfo) + { + Debug.Assert(newValueSize == sizeInfo.FieldInfo.ValueDataSize, $"Mismatched value size; expected {sizeInfo.FieldInfo.ValueDataSize}, actual {newValueSize}"); + return TrySetValueLength(ref sizeInfo); + } + + /// + /// Tries to set the length of the value field, with consideration to whether there is also space for the optionals (ETag and Expiration). + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TrySetValueLength(ref RecordSizeInfo sizeInfo) + { + var valueAddress = ValueAddress; + var oldTotalInlineValueSize = LogField.GetInlineTotalSizeOfField(valueAddress, Info.ValueIsInline); + var newTotalInlineValueSize = sizeInfo.InlineTotalValueSize; + + // Growth and fillerLen may be negative if shrinking. + var inlineValueGrowth = newTotalInlineValueSize - oldTotalInlineValueSize; + var newOptionalSize = sizeInfo.OptionalSize; + var oldOptionalSize = OptionalLength; + var inlineTotalGrowth = inlineValueGrowth + (newOptionalSize - oldOptionalSize); + + var optionalStartAddress = valueAddress + oldTotalInlineValueSize; + 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) + 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; + + // 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. + + // 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; + } + 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; + } + else if (Info.ValueIsObject && sizeInfo.ValueIsObject) + { + // Both are object records, so nothing to change. + goto Done; + } + else + { + // Overflow/Object-ness differs and we've verified there is enough space for the change, so convert. The LogField.ConvertTo* functions copy + // existing data, as we are likely here for IPU or for the initial update going from inline to overflow with Value length == sizeof(IntPtr). + if (Info.ValueIsInline) + { + 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); + } + 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); + } + } + 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); + } + 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); + } + } + else + { + 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); + } + 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); + } + } + } + + Done: + 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}"); + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TrySetValueSpan(ReadOnlySpan value, ref RecordSizeInfo sizeInfo) + { + RecordSizeInfo.AssertValueDataLength(value.Length, ref sizeInfo); + + if (!TrySetValueLength(ref sizeInfo)) + return false; + + value.CopyTo(LogField.AsSpan(ValueAddress, Info.ValueIsInline, objectIdMap)); + return true; + } + + /// + /// Set the object, checking for conversion from inline and for space for optionals (ETag, Expiration). + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] +#pragma warning disable IDE0251 // Make member 'readonly': Not doing so because it modifies internal state + public bool TrySetValueObject(IHeapObject value, ref RecordSizeInfo sizeInfo) +#pragma warning restore IDE0251 + { + return TrySetValueLength(ref 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. + /// + [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 + { + Debug.Assert(Info.ValueIsObject, $"Cannot call this overload of {GetCurrentMethodName()} for non-object Value"); + + if (Info.ValueIsInline) + { + Debug.Fail($"Cannot call {GetCurrentMethodName()} with no {nameof(RecordSizeInfo)} when the value is inline"); + return false; + } + + var objectId = *ValueObjectIdAddress; + if (objectId == ObjectIdMap.InvalidObjectId) + objectId = *ValueObjectIdAddress = objectIdMap.Allocate(); + + objectIdMap.Set(objectId, value); + return true; + } + + private readonly int ETagLen => Info.HasETag ? ETagSize : 0; + private readonly int ExpirationLen => Info.HasExpiration ? ExpirationSize : 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() + { + var actualSize = ActualRecordSize; + return (actualSize, actualSize + GetFillerLength()); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly long GetOptionalStartAddress() + { + var valueAddress = ValueAddress; + return ValueAddress + LogField.GetInlineTotalSizeOfField(valueAddress, Info.ValueIsInline); + } + + public readonly int OptionalLength => ETagLen + ExpirationLen; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly long GetETagAddress() => GetOptionalStartAddress(); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly long GetExpirationAddress() => GetETagAddress() + ETagLen; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly long GetFillerLengthAddress() => physicalAddress + ActualRecordSize; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly int GetFillerLength() => GetFillerLength(GetFillerLengthAddress()); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal readonly int GetFillerLength(long fillerLenAddress) + { + if (Info.HasFiller) + 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) + // due to RoundUp of record size. Optimize the filler address to avoid additional "if" statements. + var recSize = (int)(fillerLenAddress - physicalAddress); + return RoundUp(recSize, Constants.kRecordAlignment) - recSize; + } + + [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 + { + // 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; + + if (fillerSize >= FillerLengthSize) + { + InfoRef.SetHasFiller(); // must do this first, for zero-init + *(int*)fillerAddress = fillerSize; + } + } + + /// + /// Called during cleanup of a record allocation, before the key was copied. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void InitializeForReuse(ref 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); + + // Anything remaining is filler. + SetFillerLength(sizeInfo.AllocatedInlineRecordSize); + } + + [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 + { + if (Info.HasETag) + { + *(long*)GetETagAddress() = eTag; + return true; + } + + // We're adding an ETag where there wasn't one before. + const int growth = ETagSize; + var recordLen = ActualRecordSize; + var fillerLenAddress = physicalAddress + recordLen; + var fillerLen = GetFillerLength(fillerLenAddress) - growth; + if (fillerLen < 0) + return false; + + // 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) + *(int*)address = 0; + + // - Preserve Expiration if present; set ETag; re-enter Expiration if present + var expiration = 0L; + if (Info.HasExpiration) + { + address -= ExpirationSize; + expiration = *(long*)address; + } + + *(long*)address = eTag; + InfoRef.SetHasETag(); + address += ETagSize; + + if (Info.HasExpiration) + { + *(long*)address = expiration; // will be 0 or a valid expiration + address += ExpirationSize; // repositions to fillerAddress + } + + // - Set the new (reduced) FillerLength if there is still space for it. + if (fillerLen >= FillerLengthSize) + *(int*)address = fillerLen; + else + InfoRef.ClearHasFiller(); + return true; + } + + [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 + { + if (!Info.HasETag) + return true; + + const int growth = -ETagSize; + var recordLen = ActualRecordSize; + var fillerLenAddress = physicalAddress + recordLen; + var fillerLen = GetFillerLength(fillerLenAddress) - growth; // This will be negative, so adds ETagSize to it + + // 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). + if (Info.HasFiller) + *(int*)address = 0; + // - Move Expiration, if present, up to cover ETag; then clear the ETag bit + var expiration = 0L; + var expirationSize = 0; + if (Info.HasExpiration) + { + expirationSize = ExpirationSize; + address -= expirationSize; + expiration = *(long*)address; + *(long*)address = 0L; // To ensure zero-init + } + + // Expiration will be either zero or a valid expiration, and we have not changed the info.HasExpiration flag + address -= ETagSize; + *(long*)address = expiration; // will be 0 or a valid expiration + address += expirationSize; // repositions to fillerAddress if expirationSize is nonzero + InfoRef.ClearHasETag(); + + // - 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 + *(int*)address = fillerLen; + } + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TrySetExpiration(long expiration) + { + if (expiration == 0) + return RemoveExpiration(); + + if (Info.HasExpiration) + { + *(long*)GetExpirationAddress() = expiration; + return true; + } + + // We're adding an Expiration where there wasn't one before. + const int growth = ExpirationSize; + var recordLen = ActualRecordSize; + var fillerLenAddress = physicalAddress + recordLen; + var fillerLen = GetFillerLength(fillerLenAddress) - growth; + if (fillerLen < 0) + return false; + + // 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). + // - We must do this here in case there is not enough room for filler after the growth. + if (Info.HasFiller) + *(int*)fillerLenAddress = 0; + + // - Set Expiration where filler space used to be + InfoRef.SetHasExpiration(); + *(long*)fillerLenAddress = expiration; + fillerLenAddress += ExpirationSize; + + // - Set the new (reduced) FillerLength if there is still space for it. + if (fillerLen >= FillerLengthSize) + *(int*)fillerLenAddress = fillerLen; + else + InfoRef.ClearHasFiller(); + return true; + } + + [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 + { + if (!Info.HasExpiration) + return true; + + const int growth = -ETagSize; + var recordLen = ActualRecordSize; + var fillerLenAddress = physicalAddress + recordLen; + var fillerLen = GetFillerLength(fillerLenAddress) - growth; // This will be negative, so adds ExpirationSize to it + + // 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) + *(int*)fillerLenAddress = 0; + + // - Remove Expiration and clear the Expiration bit; this will be the new fillerLenAddress + fillerLenAddress -= ExpirationSize; + *(long*)fillerLenAddress = 0; + InfoRef.ClearHasExpiration(); + + // - 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 + *(int*)fillerLenAddress = fillerLen; + } + return true; + } + + /// + /// Copy the entire record values: Value and optionals (ETag, Expiration) + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TryCopyFrom(ref TSourceLogRecord srcLogRecord, ref RecordSizeInfo sizeInfo) + where TSourceLogRecord : ISourceLogRecord + { + // This assumes the Key has been set and is not changed + if (!srcLogRecord.Info.ValueIsObject) + { + if (!TrySetValueLength(ref sizeInfo)) + return false; + srcLogRecord.ValueSpan.CopyTo(ValueSpan); + } + else + { + Debug.Assert(srcLogRecord.ValueObject is not null, "Expected srcLogRecord.ValueObject to be set (or deserialized) already"); + if (!TrySetValueObject(srcLogRecord.ValueObject, ref sizeInfo)) + return false; + } + + return TryCopyOptionals(ref srcLogRecord, ref sizeInfo); + } + + /// + /// Copy the record optional values (ETag, Expiration) + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TryCopyOptionals(ref TSourceLogRecord srcLogRecord, ref RecordSizeInfo sizeInfo) + where TSourceLogRecord : ISourceLogRecord + { + var srcRecordInfo = srcLogRecord.Info; + + // If the source has optionals and the destination wants them, copy them over + if (!srcRecordInfo.HasETag || !sizeInfo.FieldInfo.HasETag) + _ = RemoveETag(); + else if (!TrySetETag(srcLogRecord.ETag)) + return false; + + if (!srcRecordInfo.HasExpiration || !sizeInfo.FieldInfo.HasExpiration) + _ = RemoveExpiration(); + else if (!TrySetExpiration(srcLogRecord.Expiration)) + return false; + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void ClearOptionals() + { + _ = RemoveExpiration(); + _ = RemoveETag(); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal bool FreeKeyOverflow() + { + if (!Info.KeyIsOverflow) + return false; + LogField.FreeObjectIdAndConvertToInline(ref InfoRef, KeyAddress, objectIdMap, isKey: true); + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal bool FreeValueOverflow() + { + if (!Info.ValueIsOverflow) + return false; + LogField.FreeObjectIdAndConvertToInline(ref InfoRef, ValueAddress, objectIdMap, isKey: false); + return true; + } + + /// + /// For revivification or reuse: prepare the current record to be passed to initial updaters, based upon the sizeInfo's key and value lengths. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public 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. + 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"); + + var keyLength = LogField.GetInlineLengthRef(keyAddress); + var valueLength = LogField.GetInlineLengthRef(ValueAddress); + + Debug.Assert(newKeySize + newValueSize <= allocatedSize, "Insufficient new record size"); // Should not happen as we passed sizeInfo to BlockAllocate + + // First zero out the filler. + if (Info.HasFiller) + { + *(int*)fillerAddress = 0; + InfoRef.ClearHasFiller(); + } + + // 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) + { + // 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; + } + + // Anything remaining is filler. + SetFillerLength(allocatedSize); + } + + public override readonly string ToString() + { + if (physicalAddress == 0) + return ""; + static string bstr(bool value) => value ? "T" : "F"; + var valueString = Info.ValueIsObject ? $"obj:{ValueObject}" : ValueSpan.ToShortString(20); + 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/MallocFixedPageSize.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs index 153cd3f257b..35ff1d6baba 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs @@ -13,46 +13,6 @@ namespace Tsavorite.core { - internal sealed class CountdownWrapper - { - // Separate event for sync code and tcs for async code: Do not block on async code. - private readonly CountdownEvent syncEvent; - private readonly TaskCompletionSource asyncTcs; - int remaining; - - internal CountdownWrapper(int count, bool isAsync) - { - if (isAsync) - { - asyncTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - remaining = count; - return; - } - syncEvent = new CountdownEvent(count); - } - - internal bool IsCompleted => syncEvent is null ? remaining == 0 : syncEvent.IsSet; - - internal void Wait() => syncEvent.Wait(); - internal async ValueTask WaitAsync(CancellationToken cancellationToken) - { - using var reg = cancellationToken.Register(() => asyncTcs.TrySetCanceled()); - await asyncTcs.Task.ConfigureAwait(false); - } - - internal void Decrement() - { - if (asyncTcs is not null) - { - Debug.Assert(remaining > 0); - if (Interlocked.Decrement(ref remaining) == 0) - asyncTcs.TrySetResult(0); - return; - } - syncEvent.Signal(); - } - } - /// /// Memory allocator for objects /// diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/MemoryPageScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/MemoryPageScanIterator.cs deleted file mode 100644 index e07936553d7..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/MemoryPageScanIterator.cs +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; - -namespace Tsavorite.core -{ - /// - /// Lightweight iterator for memory page (copied to buffer). GetNext() can be used outside epoch protection and locking, - /// but ctor must be called within epoch protection. - /// - /// - /// - internal sealed class MemoryPageScanIterator : ITsavoriteScanIterator - { - readonly AllocatorRecord[] page; - readonly long pageStartAddress; - readonly int recordSize; - readonly int start, end; - int offset; - - public MemoryPageScanIterator(AllocatorRecord[] page, int start, int end, long pageStartAddress, int recordSize) - { - this.page = new AllocatorRecord[page.Length]; - Array.Copy(page, start, this.page, start, end - start); - offset = start - 1; - this.start = start; - this.end = end; - this.pageStartAddress = pageStartAddress; - this.recordSize = recordSize; - } - - public long CurrentAddress => pageStartAddress + offset * recordSize; - - public long NextAddress => pageStartAddress + (offset + 1) * recordSize; - - public long BeginAddress => pageStartAddress + start * recordSize; - - public long EndAddress => pageStartAddress + end * recordSize; - - public void Dispose() - { - } - - public ref TKey GetKey() => ref page[offset].key; - public ref TValue GetValue() => ref page[offset].value; - - public bool GetNext(out RecordInfo recordInfo) - { - while (true) - { - offset++; - if (offset >= end) - { - recordInfo = default; - return false; - } - if (!page[offset].info.Invalid) - break; - } - - recordInfo = page[offset].info; - return true; - } - - public bool GetNext(out RecordInfo recordInfo, out TKey key, out TValue value) - { - var r = GetNext(out recordInfo); - if (r) - { - key = page[offset].key; - value = page[offset].value; - } - else - { - key = default; - value = default; - } - return r; - } - - /// - public override string ToString() => $"BA {BeginAddress}, EA {EndAddress}, CA {CurrentAddress}, NA {NextAddress}, start {start}, end {end}, recSize {recordSize}, pageSA {pageStartAddress}"; - } -} \ 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 new file mode 100644 index 00000000000..24b4ab532c4 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocator.cs @@ -0,0 +1,150 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.CompilerServices; + +namespace Tsavorite.core +{ + /// + /// Struct wrapper (for inlining) around the fixed-length Blittable allocator. + /// + public struct ObjectAllocator : IAllocator + where TStoreFunctions : IStoreFunctions + { + /// The wrapped class containing all data and most actual functionality. This must be the ONLY field in this structure so its size is sizeof(IntPtr). + private readonly ObjectAllocatorImpl _this; + + public ObjectAllocator(AllocatorSettings settings, TStoreFunctions storeFunctions) + { + // Called by TsavoriteKV via allocatorCreator; must pass a wrapperCreator to AllocatorBase + _this = new(settings, storeFunctions, @this => new ObjectAllocator(@this)); + } + + internal ObjectAllocator(object @this) + { + // Called by AllocatorBase via primary ctor wrapperCreator + _this = (ObjectAllocatorImpl)@this; + } + + /// + 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.GetStartLogicalAddress(page); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddress(page); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly void InitializeValue(long physicalAddress, ref RecordSizeInfo sizeInfo) => ObjectAllocatorImpl.InitializeValue(physicalAddress, ref sizeInfo); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => _this.GetRMWCopyRecordSize(ref srcLogRecord, ref input, varlenInput); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetRMWInitialRecordSize(key, ref input, varlenInput); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, value, ref input, varlenInput); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, value, ref input, varlenInput); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, ref inputLogRecord, ref input, varlenInput); + + /// Get record size required for a new tombstone record + public readonly RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) => _this.GetDeleteRecordSize(key); + + /// + public readonly void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) => _this.PopulateRecordSizeInfo(ref sizeInfo); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly void AllocatePage(int pageIndex) => _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); + + /// + [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() => _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); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => _this.CreateLogRecord(logicalAddress, physicalAddress); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs similarity index 66% rename from libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocatorImpl.cs rename to libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs index 05ab2ca74fb..c6c82584ee4 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/GenericAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectAllocatorImpl.cs @@ -1,167 +1,267 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +//#define READ_WRITE + using System; -using System.Collections.Generic; using System.Diagnostics; using System.IO; using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; using System.Threading; using Microsoft.Extensions.Logging; +using static Tsavorite.core.Utility; namespace Tsavorite.core { - internal sealed unsafe class GenericAllocatorImpl : AllocatorBase> - where TStoreFunctions : IStoreFunctions + internal sealed unsafe class ObjectAllocatorImpl : AllocatorBase> + where TStoreFunctions : IStoreFunctions { - // Circular buffer definition - internal AllocatorRecord[][] values; + /// 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; - // Object log related variables - private readonly IDevice objectLogDevice; - // Size of object chunks being written to storage - private readonly int ObjectBlockSize = 100 * (1 << 20); - // Tail offsets per segment, in object log - public readonly long[] segmentOffsets; + /// 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 + { + internal readonly ObjectIdMap objectIdMap { get; init; } + + public ObjectPage() => objectIdMap = new(); + + internal readonly void Clear() => objectIdMap?.Clear(); // TODO: Ensure we have already called the RecordDisposer + } + + internal ObjectPage[] values; - // Record sizes. We do not support variable-length keys in GenericAllocator - internal static int KeySize => Unsafe.SizeOf(); - internal static int ValueSize => Unsafe.SizeOf(); - internal static int RecordSize => Unsafe.SizeOf>(); + readonly int maxInlineKeySize; + readonly int maxInlineValueSize; - private readonly OverflowPool[]> overflowPagePool; + // Size of object chunks being written to storage + // TODO: private readonly int objectBlockSize = 100 * (1 << 20); - public GenericAllocatorImpl(AllocatorSettings settings, TStoreFunctions storeFunctions, Func> wrapperCreator) + 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) { - overflowPagePool = new OverflowPool[]>(4); + IsObjectAllocator = true; - if (settings.LogSettings.ObjectLogDevice == null) - throw new TsavoriteException("LogSettings.ObjectLogDevice needs to be specified (e.g., use Devices.CreateLogDevice, AzureStorageDevice, or NullDevice)"); + maxInlineKeySize = 1 << settings.LogSettings.MaxInlineKeySizeBits; + maxInlineValueSize = 1 << settings.LogSettings.MaxInlineValueSizeBits; - if (typeof(TKey) == typeof(SpanByte)) - throw new TsavoriteException("SpanByte Keys cannot be mixed with object Values"); - if (typeof(TValue) == typeof(SpanByte)) - throw new TsavoriteException("SpanByte Values cannot be mixed with object Keys"); + freePagePool = new OverflowPool>(4, p => { }); - values = new AllocatorRecord[BufferSize][]; - segmentOffsets = new long[SegmentBufferSize]; + var bufferSizeInBytes = (nuint)RoundUp(sizeof(long*) * BufferSize, Constants.kCacheLineBytes); + pagePointers = (long*)NativeMemory.AlignedAlloc(bufferSizeInBytes, Constants.kCacheLineBytes); + NativeMemory.Clear(pagePointers, bufferSizeInBytes); - objectLogDevice = settings.LogSettings.ObjectLogDevice; + values = new ObjectPage[BufferSize]; + for (var ii = 0; ii < BufferSize; ++ii) + values[ii] = new(); - if ((settings.LogSettings.LogDevice as NullDevice) == null && (KeyHasObjects() || ValueHasObjects())) - { - if (objectLogDevice == null) - throw new TsavoriteException("Objects in key/value, but object log not provided during creation of Tsavorite instance"); - if (objectLogDevice.SegmentSize != -1) - throw new TsavoriteException("Object log device should not have fixed segment size. Set preallocateFile to false when calling CreateLogDevice for object log"); - } + // 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"); } - internal int OverflowPageCount => overflowPagePool.Count; + internal int OverflowPageCount => freePagePool.Count; public override void Reset() { base.Reset(); - objectLogDevice.Reset(); - for (int index = 0; index < BufferSize; index++) + for (var index = 0; index < BufferSize; index++) { if (IsAllocated(index)) FreePage(index); } - Array.Clear(segmentOffsets, 0, segmentOffsets.Length); Initialize(); } + /// Allocate memory page, pinned in memory, and in sector aligned form, if possible + internal void AllocatePage(int index) + { + IncrementAllocatedPageCount(); + + 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; + } + + // No free pages are available so allocate new + pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); + values[index] = new(); + } + void ReturnPage(int index) { Debug.Assert(index < BufferSize); - if (values[index] != default) + if (pagePointers[index] != default) { - _ = overflowPagePool.TryAdd(values[index]); - values[index] = default; + _ = freePagePool.TryAdd(new() + { + pointer = pagePointers[index], + value = values[index] + }); + pagePointers[index] = default; _ = Interlocked.Decrement(ref AllocatedPageCount); } } - public override void Initialize() => Initialize(RecordSize); - - /// Get start logical address - internal long GetStartLogicalAddress(long page) => page << LogPageSizeBits; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + int GetPageIndex(long logicalAddress) => (int)((logicalAddress >> LogPageSizeBits) & (BufferSize - 1)); - /// Get first valid logical address - internal long GetFirstValidLogicalAddress(long page) - { - if (page == 0) - return (page << LogPageSizeBits) + RecordSize; - return page << LogPageSizeBits; - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal LogRecord CreateLogRecord(long logicalAddress) => CreateLogRecord(logicalAddress, GetPhysicalAddress(logicalAddress)); - internal ref RecordInfo GetInfo(long physicalAddress) - { - // Offset within page - int offset = (int)(physicalAddress & PageSizeMask); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new(physicalAddress, values[GetPageIndex(logicalAddress)].objectIdMap); - // Index of page within the circular buffer - int pageIndex = (int)((physicalAddress >> LogPageSizeBits) & BufferSizeMask); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal ObjectIdMap GetObjectIdMap(long logicalAddress) => values[GetPageIndex(logicalAddress)].objectIdMap; - return ref values[pageIndex][offset / RecordSize].info; - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => SerializeKey(key, logicalAddress, ref logRecord, maxInlineKeySize, GetObjectIdMap(logicalAddress)); - internal ref RecordInfo GetInfoFromBytePointer(byte* ptr) => ref Unsafe.AsRef>(ptr).info; + public override void Initialize() => Initialize(Constants.kFirstValidAddress); - internal ref TKey GetKey(long physicalAddress) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void InitializeValue(long physicalAddress, ref RecordSizeInfo sizeInfo) { - // Offset within page - var offset = (int)(physicalAddress & PageSizeMask); + 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(); - // Index of page within the circular buffer - var pageIndex = (int)((physicalAddress >> LogPageSizeBits) & BufferSizeMask); + if (sizeInfo.ValueIsObject) + LogRecord.GetInfoRef(physicalAddress).SetValueIsObject(); - return ref values[pageIndex][offset / RecordSize].key; + // Either an IHeapObject or a byte[] + *LogRecord.GetValueObjectIdAddress(physicalAddress) = ObjectIdMap.InvalidObjectId; + } } - internal ref TValue GetValue(long physicalAddress) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput { - // Offset within page - var offset = (int)(physicalAddress & PageSizeMask); - - // Index of page within the circular buffer - var pageIndex = (int)((physicalAddress >> LogPageSizeBits) & BufferSizeMask); - - return ref values[pageIndex][offset / RecordSize].value; + // Used by RMW to determine the length of copy destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetRMWModifiedFieldInfo(ref srcLogRecord, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } - internal (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) => (RecordSize, RecordSize); - - public int GetValueLength(ref TValue value) => ValueSize; - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SerializeKey(ref TKey src, long physicalAddress) => GetKey(physicalAddress) = src; + public RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + { + // Used by RMW to determine the length of initial destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetRMWInitialFieldInfo(key, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } - internal (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref TKey key, ref TInput input, ref TValue value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - => (RecordSize, RecordSize, KeySize); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + { + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, value, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } - internal (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref TKey key) - => (RecordSize, RecordSize, KeySize); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + { + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, value, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } - internal int GetAverageRecordSize() => RecordSize; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + { + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, ref inputLogRecord, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } - internal int GetFixedRecordSize() => RecordSize; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) + { + // Used by Delete to determine the length of a new tombstone record. Does not require an ISessionFunctions method. + var sizeInfo = new RecordSizeInfo() + { + 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 + HasETag = false, + HasExpiration = false + } + }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } - internal (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref TKey key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - => (RecordSize, RecordSize, KeySize); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) + { + // Key + sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeyDataSize <= maxInlineKeySize; + var keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : 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; + + // Record + sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; + sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); + } - internal (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref TKey key, ref TValue value) => (RecordSize, RecordSize, KeySize); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) + { + logRecord.ClearOptionals(); + if (disposeReason != DisposeReason.Deleted) + _ = logRecord.FreeKeyOverflow(); - internal (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref TKey key, ref TValue value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - => (RecordSize, RecordSize, KeySize); + if (!logRecord.Info.ValueIsInline) + { + 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); + } + } + } - internal override bool TryComplete() + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) { - var b1 = objectLogDevice.TryComplete(); - var b2 = base.TryComplete(); - return b1 || b2; + // Clear the IHeapObject if we deserialized it + if (logRecord.Info.ValueIsObject && logRecord.ValueObject is not null) + storeFunctions.DisposeValueObject(logRecord.ValueObject, disposeReason); } /// @@ -169,67 +269,53 @@ internal override bool TryComplete() /// public override void Dispose() { - if (values != null) + var localValues = Interlocked.Exchange(ref values, null); + if (localValues != null) { - for (int i = 0; i < values.Length; i++) - values[i] = null; - values = null; - } - overflowPagePool.Dispose(); - base.Dispose(); - } + base.Dispose(); + freePagePool.Dispose(); + foreach (var value in localValues) + value.Clear(); - /// Delete in-memory portion of the log - internal override void DeleteFromMemory() - { - for (int i = 0; i < values.Length; i++) - values[i] = null; - values = null; + 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; + } + } } - internal AddressInfo* GetKeyAddressInfo(long physicalAddress) - => (AddressInfo*)Unsafe.AsPointer(ref Unsafe.AsRef>((byte*)physicalAddress).key); - - internal AddressInfo* GetValueAddressInfo(long physicalAddress) - => (AddressInfo*)Unsafe.AsPointer(ref Unsafe.AsRef>((byte*)physicalAddress).value); - - /// Allocate memory page, pinned in memory, and in sector aligned form, if possible - internal void AllocatePage(int index) => values[index] = AllocatePage(); - - internal AllocatorRecord[] AllocatePage() + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public long GetPhysicalAddress(long logicalAddress) { - IncrementAllocatedPageCount(); - - if (overflowPagePool.TryGet(out var item)) - return item; + // Offset within page + var offset = (int)(logicalAddress & ((1L << LogPageSizeBits) - 1)); - return new AllocatorRecord[(PageSize + RecordSize - 1) / RecordSize]; + // Index of page within the circular buffer + var pageIndex = (int)((logicalAddress >> LogPageSizeBits) & (BufferSize - 1)); + return *(pagePointers + pageIndex) + offset; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static long SnapToLogicalAddressBoundary(ref long logicalAddress) - => logicalAddress = ((logicalAddress - Constants.kFirstValidAddress) / RecordSize) * RecordSize + Constants.kFirstValidAddress; - - public long GetPhysicalAddress(long logicalAddress) => logicalAddress; - - internal bool IsAllocated(int pageIndex) => values[pageIndex] != null; + internal bool IsAllocated(int pageIndex) => pagePointers[pageIndex] != 0; - protected override void TruncateUntilAddress(long toAddress) + protected override void TruncateUntilAddress(long toAddress) // TODO: ObjectAllocator specifics if any { base.TruncateUntilAddress(toAddress); - objectLogDevice.TruncateUntilSegment((int)(toAddress >> LogSegmentSizeBits)); } - protected override void TruncateUntilAddressBlocking(long toAddress) + protected override void TruncateUntilAddressBlocking(long toAddress) // TODO: ObjectAllocator specifics if any { base.TruncateUntilAddressBlocking(toAddress); - objectLogDevice.TruncateUntilSegment((int)(toAddress >> LogSegmentSizeBits)); } - protected override void RemoveSegment(int segment) + protected override void RemoveSegment(int segment) // TODO: ObjectAllocator specifics if any { base.RemoveSegment(segment); - objectLogDevice.RemoveSegment(segment); } protected override void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) @@ -238,13 +324,14 @@ protected override void WriteAsync(long flushPage, DeviceIOCompletionC (ulong)(AlignedPageSizeBytes * flushPage), (uint)PageSize, callback, - asyncResult, device, objectLogDevice); + asyncResult, device); } protected override void WriteAsyncToDevice (long startPage, long flushPage, int pageSize, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult, IDevice device, IDevice objectLogDevice, long[] localSegmentOffsets, long fuzzyStartLogicalAddress) { +#if READ_WRITE VerifyCompatibleSectorSize(device); VerifyCompatibleSectorSize(objectLogDevice); @@ -275,10 +362,14 @@ protected override void WriteAsyncToDevice if (epochTaken) epoch.Suspend(); } +#endif // READ_WRITE } internal void ClearPage(long page, int offset) - => Array.Clear(values[page % BufferSize], offset / RecordSize, values[page % BufferSize].Length - offset / RecordSize); + { + // 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) { @@ -288,11 +379,13 @@ internal void FreePage(long page) var thisCloseSegment = page >> (LogSegmentSizeBits - LogPageSizeBits); var nextCloseSegment = (page + 1) >> (LogSegmentSizeBits - LogPageSizeBits); +#if READ_WRITE if (thisCloseSegment != nextCloseSegment) { // We are clearing the last page in current segment segmentOffsets[thisCloseSegment % SegmentBufferSize] = 0; } +#endif // READ_WRITE // If all pages are being used (i.e. EmptyPageCount == 0), nothing to re-utilize by adding // to overflow pool. @@ -302,8 +395,9 @@ internal void FreePage(long page) private void WriteAsync(long flushPage, ulong alignedDestinationAddress, uint numBytesToWrite, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult, - IDevice device, IDevice objlogDevice, long intendedDestinationPage = -1, long[] localSegmentOffsets = null, long fuzzyStartLogicalAddress = long.MaxValue) + IDevice device, long intendedDestinationPage = -1, long[] localSegmentOffsets = null, long fuzzyStartLogicalAddress = long.MaxValue) { +#if READ_WRITE // Short circuit if we are using a null device if ((device as NullDevice) != null) { @@ -333,7 +427,7 @@ private void WriteAsync(long flushPage, ulong alignedDestinationAddres if (epoch.ThisInstanceProtected()) { epochProtected = true; - src = new AllocatorRecord[values[flushPage % BufferSize].Length]; + src = new AllocatorRecord[values[flushPage % BufferSize].Length]; Array.Copy(values[flushPage % BufferSize], src, values[flushPage % BufferSize].Length); epoch.Suspend(); } @@ -397,7 +491,7 @@ private void WriteAsync(long flushPage, ulong alignedDestinationAddres byte* recordPtr = buffer.aligned_pointer + i * RecordSize; // Retrieve reference to record struct - ref var record = ref Unsafe.AsRef>(recordPtr); + ref var record = ref Unsafe.AsRef(recordPtr); AddressInfo* key_address = null, value_address = null; // Zero out object reference addresses (AddressInfo) in the planned disk image @@ -463,7 +557,7 @@ private void WriteAsync(long flushPage, ulong alignedDestinationAddres } // 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) + if (endPosition > objectBlockSize || i == (end / RecordSize) - 1) { var memoryStreamActualLength = ms.Position; var memoryStreamTotalLength = (int)endPosition; @@ -558,6 +652,7 @@ private void WriteAsync(long flushPage, ulong alignedDestinationAddres if (epochProtected) epoch.Resume(); } +#endif // READ_WRITE } private void AsyncReadPageCallback(uint errorCode, uint numBytes, object context) @@ -574,6 +669,7 @@ protected override void ReadAsync( ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, DeviceIOCompletionCallback callback, PageAsyncReadResult asyncResult, IDevice device, IDevice objlogDevice) { +#if READ_WRITE asyncResult.freeBuffer1 = bufferPool.Get((int)aligned_read_length); asyncResult.freeBuffer1.required_bytes = (int)aligned_read_length; @@ -595,6 +691,7 @@ protected override void ReadAsync( device.ReadAsync(alignedSourceAddress, (IntPtr)asyncResult.freeBuffer1.aligned_pointer, aligned_read_length, AsyncReadPageWithObjectsCallback, asyncResult); +#endif // READ_WRITE } @@ -616,17 +713,18 @@ private void AsyncFlushPartialObjectLogCallback(uint errorCode, uint n 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; + AllocatorRecord[] src; // We are reading into a frame if (result.frame != null) { - var frame = (GenericFrame)result.frame; + var frame = (GenericFrame)result.frame; src = frame.GetPage(result.page % frame.frameSize); } else @@ -657,7 +755,7 @@ private void AsyncReadPageWithObjectsCallback(uint errorCode, uint num } // 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); + 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); @@ -674,6 +772,7 @@ private void AsyncReadPageWithObjectsCallback(uint errorCode, uint num (int)((result.page - result.offset) >> (LogSegmentSizeBits - LogPageSizeBits)), (ulong)startptr, (IntPtr)objBuffer.aligned_pointer, (uint)alignedLength, AsyncReadPageWithObjectsCallback, result); +#endif // READ_WRITE } /// @@ -685,8 +784,9 @@ private void AsyncReadPageWithObjectsCallback(uint errorCode, uint num /// /// /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) + 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); @@ -698,7 +798,7 @@ protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int num record.available_bytes = (int)(alignedReadLength - (fileOffset - alignedFileOffset)); record.required_bytes = numBytes; - var asyncResult = default(AsyncGetFromDiskResult>); + var asyncResult = default(AsyncGetFromDiskResult); asyncResult.context = context; asyncResult.context.record = result; asyncResult.context.objBuffer = record; @@ -709,73 +809,16 @@ protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int num alignedReadLength, callback, asyncResult); +#endif // READ_WRITE } - /// - /// Read pages from specified device - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - internal void AsyncReadPagesFromDeviceToFrame( - long readPageStart, - int numPages, - long untilAddress, - DeviceIOCompletionCallback callback, - TContext context, - GenericFrame frame, - out CountdownEvent completed, - long devicePageOffset = 0, - IDevice device = null, IDevice objectLogDevice = null) + public struct AllocatorRecord // TODO remove { - var usedDevice = device ?? this.device; - IDevice usedObjlogDevice = objectLogDevice; - - completed = new CountdownEvent(numPages); - for (long readPage = readPageStart; readPage < (readPageStart + numPages); readPage++) - { - int pageIndex = (int)(readPage % frame.frameSize); - if (frame.GetPage(pageIndex) == null) - frame.Allocate(pageIndex); - else - frame.Clear(pageIndex); - - var asyncResult = new PageAsyncReadResult() - { - page = readPage, - context = context, - handle = completed, - maxPtr = PageSize, - frame = frame, - }; - - var offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); - var readLength = (uint)AlignedPageSizeBytes; - long adjustedUntilAddress = (AlignedPageSizeBytes * (untilAddress >> LogPageSizeBits) + (untilAddress & PageSizeMask)); - - if (adjustedUntilAddress > 0 && ((adjustedUntilAddress - (long)offsetInFile) < PageSize)) - { - readLength = (uint)(adjustedUntilAddress - (long)offsetInFile); - asyncResult.maxPtr = readLength; - readLength = (uint)((readLength + (sectorSize - 1)) & ~(sectorSize - 1)); - } - - if (device != null) - offsetInFile = (ulong)(AlignedPageSizeBytes * (readPage - devicePageOffset)); - - ReadAsync(offsetInFile, pageIndex, readLength, callback, asyncResult, usedDevice, usedObjlogDevice); - } + public RecordInfo info; + public byte[] key; + public byte[] value; } - #region Page handlers for objects /// /// Deseialize part of page from stream @@ -785,8 +828,9 @@ internal void AsyncReadPagesFromDeviceToFrame( /// Until pointer /// /// Stream - public void Deserialize(byte* raw, long ptr, long untilptr, AllocatorRecord[] src, Stream 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; @@ -796,7 +840,7 @@ public void Deserialize(byte* raw, long ptr, long untilptr, AllocatorRecord>(raw + ptr); + ref var record = ref Unsafe.AsRef(raw + ptr); src[ptr / RecordSize].info = record.info; if (start_offset == -1) start_offset = (int)(ptr / RecordSize); @@ -841,9 +885,10 @@ public void Deserialize(byte* raw, long ptr, long untilptr, AllocatorRecord(src, start_offset, end_offset, -1, RecordSize); + using var iter = new MemoryPageScanIterator(src, start_offset, end_offset, -1, RecordSize); OnDeserializationObserver.OnNext(iter); } +#endif // READ_WRITE } /// @@ -857,13 +902,14 @@ public void Deserialize(byte* raw, long ptr, long untilptr, AllocatorRecord 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); + ref var record = ref Unsafe.AsRef(raw + ptr); if (!record.info.Invalid) { @@ -914,163 +960,108 @@ public void GetObjectInfo(byte* raw, ref long ptr, long untilptr, int objectBloc startptr = minObjAddress; size = maxObjAddress - minObjAddress; +#else + startptr = 0; + size = 0; +#endif // READ_WRITE } - - /// Retrieve objects from object log - internal bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) - { - if (!KeyHasObjects()) - ctx.key = Unsafe.AsRef>(record).key; - if (!ValueHasObjects()) - ctx.value = Unsafe.AsRef>(record).value; - - if (!(KeyHasObjects() || ValueHasObjects())) - return true; - - if (ctx.objBuffer == null) - { - // Issue IO for objects - long startAddress = -1; - long endAddress = -1; - if (KeyHasObjects()) - { - var x = GetKeyAddressInfo((long)record); - startAddress = x->Address; - endAddress = x->Address + x->Size; - } - - if (ValueHasObjects() && !GetInfoFromBytePointer(record).Tombstone) - { - var x = GetValueAddressInfo((long)record); - if (startAddress == -1) - startAddress = x->Address; - endAddress = x->Address + x->Size; - } - - // We are limited to a 2GB size per key-value - if (endAddress - startAddress > int.MaxValue) - throw new TsavoriteException("Size of key-value exceeds max of 2GB: " + (endAddress - startAddress)); - - if (startAddress < 0) - startAddress = 0; - - AsyncGetFromDisk(startAddress, (int)(endAddress - startAddress), ctx, ctx.record); - return false; - } - - // Parse the key and value objects - var ms = new MemoryStream(ctx.objBuffer.buffer); - _ = ms.Seek(ctx.objBuffer.offset + ctx.objBuffer.valid_offset, SeekOrigin.Begin); - - if (KeyHasObjects()) - { - var keySerializer = _storeFunctions.BeginDeserializeKey(ms); - keySerializer.Deserialize(out ctx.key); - keySerializer.EndDeserialize(); - } - - if (ValueHasObjects() && !GetInfoFromBytePointer(record).Tombstone) - { - var valueSerializer = _storeFunctions.BeginDeserializeValue(ms); - valueSerializer.Deserialize(out ctx.value); - valueSerializer.EndDeserialize(); - } - - ctx.objBuffer.Return(); - return true; - } - - /// Whether KVS has keys to serialize/deserialize - internal bool KeyHasObjects() => _storeFunctions.HasKeySerializer; - - /// Whether KVS has values to serialize/deserialize - internal bool ValueHasObjects() => _storeFunctions.HasValueSerializer; #endregion - public long[] GetSegmentOffsets() => segmentOffsets; + public long[] GetSegmentOffsets() => null; internal void PopulatePage(byte* src, int required_bytes, long destinationPage) - => PopulatePage(src, required_bytes, ref values[destinationPage % BufferSize]); - - internal void PopulatePageFrame(byte* src, int required_bytes, AllocatorRecord[] frame) - => PopulatePage(src, required_bytes, ref frame); + { +#if READ_WRITE + PopulatePage(src, required_bytes, ref values[destinationPage % BufferSize]); +#endif // READ_WRITE + } - internal void PopulatePage(byte* src, int required_bytes, ref AllocatorRecord[] destinationPage) + 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); } +#endif // READ_WRITE } /// /// Iterator interface for scanning Tsavorite log /// /// - public override ITsavoriteScanIterator Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords) - => new GenericScanIterator(store, this, beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch); + public override ITsavoriteScanIterator Scan(TsavoriteKV> store, + long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, bool includeSealedRecords) + => new RecordScanIterator>(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeSealedRecords: includeSealedRecords); /// /// Implementation for push-scanning Tsavorite log, called from LogAccessor /// - internal override bool Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ref TScanFunctions scanFunctions, ScanBufferingMode scanBufferingMode) + internal override bool Scan(TsavoriteKV> store, + long beginAddress, long endAddress, ref TScanFunctions scanFunctions, DiskScanBufferingMode scanBufferingMode) { - using GenericScanIterator iter = new(store, this, beginAddress, endAddress, scanBufferingMode, false, epoch, logger: logger); + using RecordScanIterator> iter = new(store, this, beginAddress, endAddress, epoch, scanBufferingMode, includeSealedRecords: false, logger: logger); return PushScanImpl(beginAddress, endAddress, ref scanFunctions, iter); } /// /// Implementation for push-scanning Tsavorite log with a cursor, called from LogAccessor /// - internal override bool ScanCursor(TsavoriteKV> store, - ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) + internal override bool ScanCursor(TsavoriteKV> store, + ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) { - using GenericScanIterator iter = new(store, this, cursor, endAddress, ScanBufferingMode.SinglePageBuffering, false, epoch, logger: logger); - return ScanLookup>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress); + using RecordScanIterator> iter = new(store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, includeSealedRecords: false, logger: logger); + return ScanLookup>>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress); } /// /// Implementation for push-iterating key versions, called from LogAccessor /// - internal override bool IterateKeyVersions(TsavoriteKV> store, - ref TKey key, long beginAddress, ref TScanFunctions scanFunctions) + internal override bool IterateKeyVersions(TsavoriteKV> store, + ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) { - using GenericScanIterator iter = new(store, this, beginAddress, epoch, logger: logger); - return IterateKeyVersionsImpl(store, ref key, beginAddress, ref scanFunctions, iter); + using RecordScanIterator> iter = new(store, this, beginAddress, epoch, logger: logger); + return IterateHashChain(store, key, beginAddress, ref scanFunctions, iter); } private void ComputeScanBoundaries(long beginAddress, long endAddress, out long pageStartAddress, out int start, out int end) { +#if READ_WRITE pageStartAddress = beginAddress & ~PageSizeMask; start = (int)(beginAddress & PageSizeMask) / RecordSize; var count = (int)(endAddress - beginAddress) / RecordSize; end = start + count; +#else + pageStartAddress = 0; + start = end = 0; +#endif // READ_WRITE } /// internal override void EvictPage(long page) { +#if READ_WRITE if (OnEvictionObserver is not null) { var beginAddress = page << LogPageSizeBits; var endAddress = (page + 1) << LogPageSizeBits; ComputeScanBoundaries(beginAddress, endAddress, out var pageStartAddress, out var start, out var end); - using var iter = new MemoryPageScanIterator(values[(int)(page % BufferSize)], start, end, pageStartAddress, RecordSize); + using var iter = new MemoryPageScanIterator(values[(int)(page % BufferSize)], start, end, pageStartAddress, RecordSize); OnEvictionObserver?.OnNext(iter); } FreePage(page); +#endif // READ_WRITE } /// - internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver> observer) + internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver observer) { +#if READ_WRITE var page = (beginAddress >> LogPageSizeBits) % BufferSize; ComputeScanBoundaries(beginAddress, endAddress, out var pageStartAddress, out var start, out var end); - using var iter = new MemoryPageScanIterator(values[page], start, end, pageStartAddress, RecordSize); + using var iter = new MemoryPageScanIterator(values[page], start, end, pageStartAddress, RecordSize); Debug.Assert(epoch.ThisInstanceProtected()); try { @@ -1081,6 +1072,7 @@ internal override void MemoryPageScan(long beginAddress, long endAddress, IObser { epoch.Resume(); } +#endif // READ_WRITE } internal override void AsyncFlushDeltaToDevice(long startAddress, long endAddress, long prevEndAddress, long version, DeltaLog deltaLog, out SemaphoreSlim completedSemaphore, int throttleCheckpointFlushDelayMs) diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs new file mode 100644 index 00000000000..48a1b8620c1 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ObjectIdMap.cs @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.CompilerServices; + +namespace Tsavorite.core +{ + /// + /// Maps the ObjectId in the ObjectAllocator's Value field to the actual object in the object multi-level array. + /// This may be either a byte[] Span-overflow allocation, or an IHeapObject. + /// + public class ObjectIdMap + { + /// We will never return a negative index from Allocate + public const int InvalidObjectId = -1; + + /// Size of the object Id + public const int ObjectIdSize = sizeof(int); + + // For this class, the "page" is an object. + internal MultiLevelPageArray objectArray; + + internal SimpleConcurrentStack freeSlots; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal ObjectIdMap() + { + // entriesPerPage comes from ObjectAllocator's minimum pagesize / expected record size so is the maximum possible number of records. + // Records may be larger due to key size but we have limits on that so it is unlikely we will waste very much of this allocation. + objectArray = new(); + freeSlots = new(); + } + + internal int Count => objectArray.Count; + + /// Reserve a slot and return its ID. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public int Allocate() + => freeSlots.TryPop(out var objectId) ? objectId : objectArray.Allocate(); + + /// 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); + 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.). + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Free(ref int objectIdRef) + { + var objectId = objectIdRef; + objectIdRef = InvalidObjectId; + Free(objectId); + } + + /// Returns the slot's object. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal object Get(int objectId) => objectArray.Get(objectId); + + /// Returns the slot's object. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void Set(int objectId, object element) => objectArray.Set(objectId, element); + + /// Clear the array. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Clear() => objectArray?.Clear(); // TODO reduce allocated chapter count also? + + /// Clear a specific slot of the array. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void ClearAt(int objectId, Action disposer) + { + var element = Get(objectId); + disposer(Unsafe.As(ref element)); + Set(objectId, default); + } + + /// + public override string ToString() => $"tail: {(objectArray is not null ? objectArray.tail.ToString() : "")}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs new file mode 100644 index 00000000000..20385e0a37b --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/OptionalFieldsShift.cs @@ -0,0 +1,69 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Runtime.CompilerServices; + +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). + /// + 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); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private void GetAndZero(long address, ref 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; + address += LogRecord.ETagSize; + } + if (recordInfo.HasExpiration) + { + Expiration = *(long*)address; + *(long*)address = 0; + 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) + { + // Restore after shift. See comments in GetAndZero for more details. + if (recordInfo.HasETag) + { + *(long*)address = ETag; + address += LogRecord.ETagSize; + } + if (recordInfo.HasExpiration) + { + *(long*)address = Expiration; + address += LogRecord.ExpirationSize; + } + if (fillerLen >= LogRecord.FillerLengthSize) + { + *(long*)address = fillerLen; + recordInfo.SetHasFiller(); + } + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/PageUnit.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/PageUnit.cs index 5d093c3fd55..715111fe467 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/PageUnit.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/PageUnit.cs @@ -5,10 +5,13 @@ namespace Tsavorite.core { - struct PageUnit + struct PageUnit { - public byte[] value; + public TValuePage value; public long pointer; + + /// + public override string ToString() => $"Value {value}, Pointer {pointer}"; } [StructLayout(LayoutKind.Explicit)] @@ -18,6 +21,9 @@ internal struct FullPageStatus public long LastFlushedUntilAddress; [FieldOffset(8)] public long Dirty; + + /// + public override string ToString() => $"LastFUA {LastFlushedUntilAddress}, Dirty {Dirty}"; } [StructLayout(LayoutKind.Explicit)] @@ -29,5 +35,8 @@ internal struct PageOffset public int Page; [FieldOffset(0)] public long PageAndOffset; + + /// + public override string ToString() => $"Page {Page}, Offset {Offset}"; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs new file mode 100644 index 00000000000..056c259be3a --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/RecordScanIterator.cs @@ -0,0 +1,401 @@ +// 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 + /// + public sealed unsafe class RecordScanIterator : 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 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 RecordScanIterator(TsavoriteKV store, AllocatorBase hlogBase, + long beginAddress, long endAddress, LightEpoch epoch, + DiskScanBufferingMode diskScanBufferingMode, InMemoryScanBufferingMode memScanBufferingMode = InMemoryScanBufferingMode.NoBuffering, + bool includeSealedRecords = false, bool assumeInMemory = false, ILogger logger = null) + : base(beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, diskScanBufferingMode, memScanBufferingMode, includeSealedRecords, 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 RecordScanIterator(TsavoriteKV store, AllocatorBase hlogBase, + long beginAddress, LightEpoch epoch, ILogger logger = null) + : base(beginAddress == 0 ? hlogBase.GetFirstValidLogicalAddress(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()); + } + + /// + 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 = 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 = currentAddress >> hlogBase.LogPageSizeBits; + 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 = logicalAddress & hlogBase.PageSizeMask; + var physicalAddress = GetPhysicalAddress(logicalAddress, headAddress, currentPage, offset, out long allocatedSize) - offset; + long totalSizes = 0; + if (currentPage == 0) + { + if (logicalAddress < hlogBase.BeginAddress) + return logicalAddress = hlogBase.BeginAddress; + physicalAddress += hlogBase.BeginAddress; + totalSizes = (int)hlogBase.BeginAddress; + } + + if (logicalAddress >= headAddress) + { + while (totalSizes <= offset) + { + if (totalSizes + allocatedSize > offset) + break; + totalSizes += allocatedSize; + physicalAddress += allocatedSize; + } + } + else + { + while (totalSizes <= offset) + { + if (totalSizes + allocatedSize > offset) + break; + totalSizes += allocatedSize; + physicalAddress += allocatedSize; + } + } + + return logicalAddress += totalSizes - offset; + } + + /// + /// 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 = currentAddress & hlogBase.PageSizeMask; + var physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset, out long allocatedSize); + + // If record did not fit on the page its recordInfo will be Null; skip to the next page if so. + var recordInfo = LogRecord.GetInfo(physicalAddress); + + if (recordInfo.IsNull) + { + nextAddress = (1 + (currentAddress >> hlogBase.LogPageSizeBits)) << hlogBase.LogPageSizeBits; + continue; + } + + nextAddress = currentAddress + allocatedSize; + + var skipOnScan = includeSealedRecords ? recordInfo.Invalid : 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. + OperationStackContext stackCtx = default; + try + { + if (currentAddress >= headAddress && store is not null) + store.LockForScan(ref stackCtx, logRecord.Key); + diskLogRecord.Serialize(in logRecord, hlogBase.bufferPool, valueSerializer: default, ref recordBuffer); + } + 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. + diskLogRecord = new(physicalAddress); + _ = diskLogRecord.DeserializeValueObject(hlogBase.storeFunctions.CreateValueObjectSerializer()); + nextAddress = currentAddress + diskLogRecord.GetSerializedLength(); + } + } + 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 = includeSealedRecords ? logRecord.Info.Invalid : 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(); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + long GetPhysicalAddress(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 bool IsSet => diskLogRecord.IsSet; + + /// + public ReadOnlySpan Key => diskLogRecord.Key; + + /// + public bool IsPinnedKey => diskLogRecord.IsPinnedKey; + + /// + public byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; + + /// + public Span ValueSpan => diskLogRecord.ValueSpan; + + /// + public IHeapObject ValueObject => diskLogRecord.ValueObject; + + /// + public ReadOnlySpan RecordSpan => diskLogRecord.RecordSpan; + + /// + public bool IsPinnedValue => diskLogRecord.IsPinnedValue; + + /// + public byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; + + /// + public long ETag => diskLogRecord.ETag; + + /// + public long Expiration => diskLogRecord.Expiration; + + /// + public void ClearValueObject(Action disposer) { } // Not relevant for iterators + + /// + public bool AsLogRecord(out LogRecord logRecord) + { + logRecord = default; + return false; + } + + /// + public bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) + { + diskLogRecord = this.diskLogRecord; + return true; + } + + /// + [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 + }; + #endregion // ISourceLogRecord + + /// + /// Dispose iterator + /// + public override void Dispose() + { + base.Dispose(); + if (diskLogRecord.IsSet) + hlogBase._wrapper.DisposeRecord(ref diskLogRecord, DisposeReason.DeserializedFromDisk); + recordBuffer?.Return(); + recordBuffer = null; + frame?.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) + => hlogBase.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice); + + private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) + { + var result = (PageAsyncReadResult)context; + + if (errorCode != 0) + { + 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)); + _ = diskLogRecord.DeserializeValueObject(hlogBase.storeFunctions.CreateValueObjectSerializer()); + + if (errorCode == 0) + _ = result.handle?.Signal(); + + Interlocked.MemoryBarrier(); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanCursorState.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanCursorState.cs index eeddbebc452..3b74a8f4fa2 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanCursorState.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanCursorState.cs @@ -3,15 +3,15 @@ namespace Tsavorite.core { - internal sealed class ScanCursorState + internal sealed class ScanCursorState { - internal IScanIteratorFunctions functions; + internal IScanIteratorFunctions functions; internal long acceptedCount; // Number of records pushed to and accepted by the caller internal bool endBatch; // End the batch (but return a valid cursor for the next batch, as if "count" records had been returned) internal bool retryLastRecord; // Retry the last record when returning a valid cursor internal bool stop; // Stop the operation (as if all records in the db had been returned) - internal void Initialize(IScanIteratorFunctions scanIteratorFunctions) + internal void Initialize(IScanIteratorFunctions scanIteratorFunctions) { functions = scanIteratorFunctions; acceptedCount = 0; diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs index a09d0e0bd4b..b645cbb93eb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/ScanIteratorBase.cs @@ -69,6 +69,8 @@ public abstract class ScanIteratorBase /// protected ILogger logger; + protected InMemoryScanBufferingMode memScanBufferingMode; + /// /// Constructor /// @@ -79,9 +81,12 @@ public abstract class ScanIteratorBase /// /// /// - public unsafe ScanIteratorBase(long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords, LightEpoch epoch, int logPageSizeBits, bool initForReads = true, ILogger logger = null) + public unsafe ScanIteratorBase(long beginAddress, long endAddress, DiskScanBufferingMode scanBufferingMode, InMemoryScanBufferingMode memScanBufferingMode, + bool includeSealedRecords, LightEpoch epoch, int logPageSizeBits, bool initForReads = true, ILogger logger = null) { this.logger = logger; + this.memScanBufferingMode = memScanBufferingMode; + // If we are protected when creating the iterator, we do not need per-GetNext protection if (epoch != null && !epoch.ThisInstanceProtected()) this.epoch = epoch; @@ -94,11 +99,11 @@ public unsafe ScanIteratorBase(long beginAddress, long endAddress, ScanBuffering currentAddress = -1; nextAddress = beginAddress; - if (scanBufferingMode == ScanBufferingMode.SinglePageBuffering) + if (scanBufferingMode == DiskScanBufferingMode.SinglePageBuffering) frameSize = 1; - else if (scanBufferingMode == ScanBufferingMode.DoublePageBuffering) + else if (scanBufferingMode == DiskScanBufferingMode.DoublePageBuffering) frameSize = 2; - else if (scanBufferingMode == ScanBufferingMode.NoBuffering) + else if (scanBufferingMode == DiskScanBufferingMode.NoBuffering) { frameSize = 0; return; diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs index 4949003aae2..6c7c9ac3924 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocator.cs @@ -1,13 +1,14 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Runtime.CompilerServices; namespace Tsavorite.core { - // Allocator for SpanByte Keys and Values. - public struct SpanByteAllocator : IAllocator - where TStoreFunctions : IStoreFunctions + // Allocator for ReadOnlySpan Keys and Span Values. + public struct SpanByteAllocator : IAllocator + where TStoreFunctions : IStoreFunctions { /// The wrapped class containing all data and most actual functionality. This must be the ONLY field in this structure so its size is sizeof(IntPtr). private readonly SpanByteAllocatorImpl _this; @@ -18,16 +19,16 @@ public SpanByteAllocator(AllocatorSettings settings, TStoreFunctions storeFuncti _this = new(settings, storeFunctions, @this => new SpanByteAllocator(@this)); } - public SpanByteAllocator(object @this) + internal SpanByteAllocator(object @this) { // Called by AllocatorBase via primary ctor wrapperCreator _this = (SpanByteAllocatorImpl)@this; } /// - public readonly AllocatorBase GetBase() - where TAllocator : IAllocator - => (AllocatorBase)(object)_this; + public readonly AllocatorBase GetBase() + where TAllocator : IAllocator + => (AllocatorBase)(object)_this; /// public readonly bool IsFixedLength => false; @@ -49,77 +50,45 @@ public readonly AllocatorBase G /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref RecordInfo GetInfo(long physicalAddress) - => ref SpanByteAllocatorImpl.GetInfo(physicalAddress); + public readonly void InitializeValue(long physicalAddress, ref RecordSizeInfo sizeInfo) => _this.InitializeValue(physicalAddress, ref sizeInfo); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe ref RecordInfo GetInfoFromBytePointer(byte* ptr) - => ref SpanByteAllocatorImpl.GetInfoFromBytePointer(ptr); + public readonly RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => _this.GetRMWCopyRecordSize(ref srcLogRecord, ref input, varlenInput); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref SpanByte GetKey(long physicalAddress) - => ref SpanByteAllocatorImpl.GetKey(physicalAddress); + public readonly RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetRMWInitialRecordSize(key, ref input, varlenInput); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref SpanByte GetValue(long physicalAddress) => ref _this.GetValue(physicalAddress); + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, value, ref input, varlenInput); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref SpanByte GetAndInitializeValue(long physicalAddress, long endPhysicalAddress) => ref _this.GetAndInitializeValue(physicalAddress, endPhysicalAddress); + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, value, ref input, varlenInput); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) => _this.GetRecordSize(physicalAddress); + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => _this.GetUpsertRecordSize(key, ref inputLogRecord, ref input, varlenInput); - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref SpanByte key, ref TInput input, ref SpanByte value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - where TVariableLengthInput : IVariableLengthInput - => _this.GetRMWCopyDestinationRecordSize(ref key, ref input, ref value, ref recordInfo, varlenInput); - - /// - public (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref SpanByte key) => _this.GetTombstoneRecordSize(ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetRequiredRecordSize(long physicalAddress, int availableBytes) => _this.GetRequiredRecordSize(physicalAddress, availableBytes); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetAverageRecordSize() => _this.GetAverageRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetFixedRecordSize() => _this.GetFixedRecordSize(); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref SpanByte key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => _this.GetRMWInitialRecordSize(ref key, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref SpanByte key, ref SpanByte value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput - => _this.GetUpsertRecordSize(ref key, ref value, ref input, sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref SpanByte key, ref SpanByte value) => _this.GetRecordSize(ref key, ref value); + /// Get record size required for a new tombstone record + public readonly RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) => _this.GetDeleteRecordSize(key); /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly int GetValueLength(ref SpanByte value) - => SpanByteAllocatorImpl.GetValueLength(ref value); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) - => SpanByteAllocatorImpl.RetrievedFullRecord(record, ref ctx); + public readonly void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) => _this.PopulateRecordSizeInfo(ref sizeInfo); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -151,32 +120,30 @@ public readonly unsafe bool RetrievedFullRecord(byte* record, ref AsyncIOContext /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref SpanByte GetContextRecordKey(ref AsyncIOContext ctx) - => ref SpanByteAllocatorImpl.GetContextRecordKey(ref ctx); + public readonly long[] GetSegmentOffsets() + => SpanByteAllocatorImpl.GetSegmentOffsets(); /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ref SpanByte GetContextRecordValue(ref AsyncIOContext ctx) => ref _this.GetContextRecordValue(ref ctx); + public readonly int OverflowPageCount => _this.OverflowPageCount; /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetKeyContainer(ref SpanByte key) => _this.GetKeyContainer(ref key); + public readonly void SerializeKey(ReadOnlySpan key, long logicalAddress, ref LogRecord logRecord) => _this.SerializeKey(key, logicalAddress, ref logRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly IHeapContainer GetValueContainer(ref SpanByte value) => _this.GetValueContainer(ref value); + public readonly LogRecord CreateLogRecord(long logicalAddress) => _this.CreateLogRecord(logicalAddress); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long[] GetSegmentOffsets() - => SpanByteAllocatorImpl.GetSegmentOffsets(); + public readonly LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => _this.CreateLogRecord(logicalAddress, physicalAddress); /// - public readonly int OverflowPageCount => _this.OverflowPageCount; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void SerializeKey(ref SpanByte key, long physicalAddress) - => SpanByteAllocatorImpl.SerializeKey(ref key, physicalAddress); + public void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) => _this.DisposeRecord(ref logRecord, disposeReason); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs index 307f0a5bc47..56f2ca25070 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteAllocatorImpl.cs @@ -4,36 +4,33 @@ using System; using System.Diagnostics; using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; using System.Threading; using static Tsavorite.core.Utility; namespace Tsavorite.core { - // Allocator for SpanByte, possibly with a Blittable Key or Value. - internal sealed unsafe class SpanByteAllocatorImpl : AllocatorBase> - where TStoreFunctions : IStoreFunctions + // Allocator for ReadOnlySpan Key and Span Value. + internal sealed unsafe class SpanByteAllocatorImpl : AllocatorBase> + where TStoreFunctions : IStoreFunctions { - // Circular buffer definition - private readonly byte[][] values; - private readonly long[] pointers; - private readonly long* nativePointers; + /// Circular buffer definition + /// The long is actually a byte*, but storing as 'long' makes going through logicalAddress/physicalAddress translation more easily + long* pagePointers; - private readonly OverflowPool overflowPagePool; + private OverflowPool> freePagePool; public SpanByteAllocatorImpl(AllocatorSettings settings, TStoreFunctions storeFunctions, Func> wrapperCreator) : base(settings.LogSettings, storeFunctions, wrapperCreator, settings.evictCallback, settings.epoch, settings.flushCallback, settings.logger) { - overflowPagePool = new OverflowPool(4, p => { }); + freePagePool = new OverflowPool>(4, p => { }); - if (BufferSize > 0) - { - values = new byte[BufferSize][]; - pointers = GC.AllocateArray(BufferSize, true); - nativePointers = (long*)Unsafe.AsPointer(ref pointers[0]); - } + var bufferSizeInBytes = (nuint)RoundUp(sizeof(long*) * BufferSize, Constants.kCacheLineBytes); + pagePointers = (long*)NativeMemory.AlignedAlloc(bufferSizeInBytes, Constants.kCacheLineBytes); + NativeMemory.Clear(pagePointers, bufferSizeInBytes); } - internal int OverflowPageCount => overflowPagePool.Count; + internal int OverflowPageCount => freePagePool.Count; public override void Reset() { @@ -46,194 +43,179 @@ public override void Reset() Initialize(); } + /// Allocate memory page, pinned in memory, and in sector aligned form, if possible + 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; + } + + // No free pages are available so allocate new + pagePointers[index] = (long)NativeMemory.AlignedAlloc((nuint)PageSize, (nuint)sectorSize); + } + 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; - Interlocked.Decrement(ref AllocatedPageCount); + pagePointers[index] = default; + _ = Interlocked.Decrement(ref AllocatedPageCount); } } - public override void Initialize() => Initialize(Constants.kFirstValidAddress); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref RecordInfo GetInfo(long physicalAddress) => ref Unsafe.AsRef((void*)physicalAddress); + int GetPageIndex(long logicalAddress) => (int)((logicalAddress >> LogPageSizeBits) & (BufferSize - 1)); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref RecordInfo GetInfoFromBytePointer(byte* ptr) => ref Unsafe.AsRef(ptr); + internal LogRecord CreateLogRecord(long logicalAddress) => CreateLogRecord(logicalAddress, GetPhysicalAddress(logicalAddress)); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref SpanByte GetKey(long physicalAddress) => ref Unsafe.AsRef((byte*)physicalAddress + RecordInfo.GetLength()); + internal LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => new LogRecord(physicalAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref SpanByte GetValue(long physicalAddress) => ref Unsafe.AsRef((byte*)ValueOffset(physicalAddress)); + 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(Constants.kFirstValidAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ref SpanByte GetAndInitializeValue(long physicalAddress, long endAddress) + public void InitializeValue(long physicalAddress, ref RecordSizeInfo sizeInfo) { - var src = (byte*)ValueOffset(physicalAddress); + // Value is always inline in the SpanByteAllocator + var valueAddress = LogRecord.GetValueAddress(physicalAddress); - // Initialize the SpanByte to the length of the entire value space, less the length of the int size prefix. - *(int*)src = (int)((byte*)endAddress - src) - sizeof(int); - return ref Unsafe.AsRef(src); + LogRecord.GetInfoRef(physicalAddress).SetValueIsInline(); + _ = LogField.SetInlineDataLength(valueAddress, sizeInfo.FieldInfo.ValueDataSize); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static long KeyOffset(long physicalAddress) => physicalAddress + RecordInfo.GetLength(); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private long ValueOffset(long physicalAddress) => KeyOffset(physicalAddress) + AlignedKeySize(physicalAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private int AlignedKeySize(long physicalAddress) => RoundUp(KeySize(physicalAddress), Constants.kRecordAlignment); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private int KeySize(long physicalAddress) => (*(SpanByte*)KeyOffset(physicalAddress)).TotalSize; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private int ValueSize(long physicalAddress) => (*(SpanByte*)ValueOffset(physicalAddress)).TotalSize; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int GetValueLength(ref SpanByte value) => value.TotalSize; - - const int FieldInitialLength = sizeof(int); // The .Length field of a SpanByte is the initial length - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) + public RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput { - ref var recordInfo = ref GetInfo(physicalAddress); - if (recordInfo.IsNull()) - return (RecordInfo.GetLength(), RecordInfo.GetLength()); - - var valueLen = ValueSize(physicalAddress); - if (recordInfo.HasFiller) // Get the extraValueLength - valueLen += *(int*)(ValueOffset(physicalAddress) + RoundUp(valueLen, sizeof(int))); - - var size = RecordInfo.GetLength() + AlignedKeySize(physicalAddress) + valueLen; - return (size, RoundUp(size, Constants.kRecordAlignment)); + // Used by RMW to determine the length of copy destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetRMWModifiedFieldInfo(ref srcLogRecord, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref SpanByte key, ref TInput input, ref SpanByte value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - where TVariableLengthInput : IVariableLengthInput + public RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput { - // Used by RMW to determine the length of copy destination (taking Input into account), so does not need to get filler length. - var keySize = key.TotalSize; - var size = RecordInfo.GetLength() + RoundUp(keySize, Constants.kRecordAlignment) + varlenInput.GetRMWModifiedValueLength(ref value, ref input); - return (size, RoundUp(size, Constants.kRecordAlignment), keySize); + // Used by RMW to determine the length of initial destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetRMWInitialFieldInfo(key, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref SpanByte key) + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput { - int keySize = key.TotalSize; - // Only metadata space needed since this is going to be used for tombstoning anyway. - int minAllocationForTombstone = sizeof(int); - int size = RecordInfo.GetLength() + RoundUp(keySize, Constants.kRecordAlignment) + minAllocationForTombstone; - return (size, RoundUp(size, Constants.kRecordAlignment), keySize); + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, value, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } - public int GetRequiredRecordSize(long physicalAddress, int availableBytes) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput { - // We need at least [average record size]... - var reqBytes = GetAverageRecordSize(); - if (availableBytes < reqBytes) - return reqBytes; - - // We need at least [RecordInfo size] + [actual key size]... - reqBytes = RecordInfo.GetLength() + AlignedKeySize(physicalAddress) + FieldInitialLength; - if (availableBytes < reqBytes) - return reqBytes; - - // We need at least [RecordInfo size] + [actual key size] + [actual value size] - var recordInfo = GetInfo(physicalAddress); - var valueLen = ValueSize(physicalAddress); - if (recordInfo.HasFiller) - { - // We have a filler, so the valueLen we have now is the usedValueLength; we need to offset to where the extraValueLength is and read that int - var alignedUsedValueLength = RoundUp(valueLen, sizeof(int)); - reqBytes = RecordInfo.GetLength() + AlignedKeySize(physicalAddress) + alignedUsedValueLength + sizeof(int); - if (availableBytes < reqBytes) - return reqBytes; - valueLen += *(int*)(ValueOffset(physicalAddress) + alignedUsedValueLength); - } - - // Now we know the full record length. - reqBytes = RecordInfo.GetLength() + AlignedKeySize(physicalAddress) + valueLen; - reqBytes = RoundUp(reqBytes, Constants.kRecordAlignment); - return reqBytes; + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, ref inputLogRecord, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetAverageRecordSize() => RecordInfo.GetLength() + (RoundUp(FieldInitialLength, Constants.kRecordAlignment) * 2); + public RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + { + // Used by Upsert to determine the length of insert destination (client uses Input to fill in whether ETag and Expiration are inluded); Filler information is not needed. + var sizeInfo = new RecordSizeInfo() { FieldInfo = varlenInput.GetUpsertFieldInfo(key, value, ref input) }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; + } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetFixedRecordSize() => GetAverageRecordSize(); - - public (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref SpanByte key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput + public RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) { - int keySize = key.TotalSize; - var actualSize = RecordInfo.GetLength() + RoundUp(keySize, Constants.kRecordAlignment) + sessionFunctions.GetRMWInitialValueLength(ref input); - return (actualSize, RoundUp(actualSize, Constants.kRecordAlignment), keySize); + // Used by Delete to determine the length of a new tombstone record. Does not require an ISessionFunctions method. + var sizeInfo = new RecordSizeInfo() + { + FieldInfo = new() + { + KeyDataSize = key.Length, + ValueDataSize = 0, // No payload for the default value + HasETag = false, + HasExpiration = false + } + }; + PopulateRecordSizeInfo(ref sizeInfo); + return sizeInfo; } - public (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref SpanByte key, ref SpanByte value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : IVariableLengthInput + public void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) { - int keySize = key.TotalSize; - var actualSize = RecordInfo.GetLength() + RoundUp(keySize, Constants.kRecordAlignment) + sessionFunctions.GetUpsertValueLength(ref value, ref input); - return (actualSize, RoundUp(actualSize, Constants.kRecordAlignment), keySize); + // For SpanByteAllocator, we are always inline. + // Key + sizeInfo.KeyIsInline = true; + var keySize = sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize; + + // Value + sizeInfo.MaxInlineValueSpanSize = int.MaxValue; // Not currently doing out-of-line for SpanByteAllocator + sizeInfo.ValueIsInline = true; + var valueSize = sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize; + + // Record + sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; + sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); } - public (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref SpanByte key, ref SpanByte value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) { - int keySize = key.TotalSize; - var actualSize = RecordInfo.GetLength() + RoundUp(keySize, Constants.kRecordAlignment) + value.TotalSize; - return (actualSize, RoundUp(actualSize, Constants.kRecordAlignment), keySize); + logRecord.ClearOptionals(); + // Key and Value are always inline in the SpanByteAllocator so this is a no-op } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static void SerializeKey(ref SpanByte src, long physicalAddress) => src.CopyTo((byte*)KeyOffset(physicalAddress)); + internal void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) { /* This allocator has no IHeapObject */ } /// /// Dispose memory allocator /// public override void Dispose() { - base.Dispose(); - overflowPagePool.Dispose(); - } - - /// - /// Allocate memory page, pinned in memory, and in sector aligned form, if possible - /// - /// - internal void AllocatePage(int index) - { - IncrementAllocatedPageCount(); - - if (overflowPagePool.TryGet(out var item)) + var localFreePagePool = Interlocked.Exchange(ref freePagePool, null); + if (localFreePagePool != null) { - pointers[index] = item.pointer; - values[index] = item.value; - return; - } - - var adjustedSize = PageSize + 2 * sectorSize; + base.Dispose(); + localFreePagePool.Dispose(); - 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; + 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; + } + } } [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -243,15 +225,15 @@ public long GetPhysicalAddress(long logicalAddress) var offset = (int)(logicalAddress & ((1L << LogPageSizeBits) - 1)); // Index of page within the circular buffer - var pageIndex = (int)((logicalAddress >> LogPageSizeBits) & (BufferSize - 1)); - return *(nativePointers + pageIndex) + offset; + var pageIndex = GetPageIndex(logicalAddress); + return *(pagePointers + pageIndex) + offset; } - internal bool IsAllocated(int pageIndex) => values[pageIndex] != null; + internal bool IsAllocated(int pageIndex) => pagePointers[pageIndex] != 0; protected override void WriteAsync(long flushPage, DeviceIOCompletionCallback callback, PageAsyncFlushResult asyncResult) { - WriteAsync((IntPtr)pointers[flushPage % BufferSize], + WriteAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * flushPage), (uint)AlignedPageSizeBytes, callback, @@ -265,32 +247,14 @@ protected override void WriteAsyncToDevice VerifyCompatibleSectorSize(device); var alignedPageSize = (pageSize + (sectorSize - 1)) & ~(sectorSize - 1); - WriteAsync((IntPtr)pointers[flushPage % BufferSize], + WriteAsync((IntPtr)pagePointers[flushPage % BufferSize], (ulong)(AlignedPageSizeBytes * (flushPage - startPage)), (uint)alignedPageSize, callback, asyncResult, device); } - public long GetStartLogicalAddress(long page) => page << LogPageSizeBits; - - public long GetFirstValidLogicalAddress(long page) - { - if (page == 0) - return (page << LogPageSizeBits) + Constants.kFirstValidAddress; - return page << LogPageSizeBits; - } - 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); - } - } + => NativeMemory.Clear((byte*)pagePointers[page % BufferSize] + offset, (nuint)(PageSize - offset)); internal void FreePage(long page) { @@ -299,21 +263,10 @@ internal void FreePage(long page) ReturnPage((int)(page % BufferSize)); } - /// - /// Delete in-memory portion of the log - /// - internal override void DeleteFromMemory() - { - for (int i = 0; i < values.Length; i++) - values[i] = null; - } - - protected override void ReadAsync( - ulong alignedSourceAddress, int destinationPageIndex, uint aligned_read_length, + 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); + device.ReadAsync(alignedSourceAddress, (IntPtr)pagePointers[destinationPageIndex], aligned_read_length, callback, asyncResult); } /// @@ -325,21 +278,11 @@ protected override void ReadAsync( /// /// /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) + protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) { throw new InvalidOperationException("AsyncReadRecordObjectsToMemory invalid for SpanByteAllocator"); } - internal static bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) => true; - - internal static ref SpanByte GetContextRecordKey(ref AsyncIOContext ctx) => ref GetKey((long)ctx.record.GetValidPointer()); - - internal ref SpanByte GetContextRecordValue(ref AsyncIOContext ctx) => ref GetValue((long)ctx.record.GetValidPointer()); - - internal IHeapContainer GetKeyContainer(ref SpanByte key) => new SpanByteHeapContainer(ref key, bufferPool); - - internal IHeapContainer GetValueContainer(ref SpanByte value) => new SpanByteHeapContainer(ref value, bufferPool); - internal static long[] GetSegmentOffsets() => null; internal void PopulatePage(byte* src, int required_bytes, long destinationPage) @@ -351,114 +294,46 @@ internal void PopulatePage(byte* src, int required_bytes, long destinationPage) /// /// Iterator interface for pull-scanning Tsavorite log /// - public override ITsavoriteScanIterator Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords) - => new SpanByteScanIterator(store, this, beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch, logger: logger); + public override ITsavoriteScanIterator Scan(TsavoriteKV> store, + long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, bool includeSealedRecords) + => new RecordScanIterator>(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, includeSealedRecords: includeSealedRecords, logger: logger); /// /// Implementation for push-scanning Tsavorite log, called from LogAccessor /// - internal override bool Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ref TScanFunctions scanFunctions, ScanBufferingMode scanBufferingMode) + internal override bool Scan(TsavoriteKV> store, + long beginAddress, long endAddress, ref TScanFunctions scanFunctions, DiskScanBufferingMode diskScanBufferingMode) { - using SpanByteScanIterator iter = new(store, this, beginAddress, endAddress, scanBufferingMode, false, epoch, logger: logger); + using RecordScanIterator> iter = new(store, this, beginAddress, endAddress, epoch, diskScanBufferingMode, logger: logger); return PushScanImpl(beginAddress, endAddress, ref scanFunctions, iter); } /// /// Implementation for push-scanning Tsavorite log with a cursor, called from LogAccessor /// - internal override bool ScanCursor(TsavoriteKV> store, - ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) + internal override bool ScanCursor(TsavoriteKV> store, + ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) { - using SpanByteScanIterator iter = new(store, this, cursor, endAddress, ScanBufferingMode.SinglePageBuffering, false, epoch, logger: logger); - return ScanLookup>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress); + using RecordScanIterator> iter = new(store, this, cursor, endAddress, epoch, DiskScanBufferingMode.SinglePageBuffering, logger: logger); + return ScanLookup>>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress); } /// /// Implementation for push-iterating key versions, called from LogAccessor /// - internal override bool IterateKeyVersions(TsavoriteKV> store, - ref SpanByte key, long beginAddress, ref TScanFunctions scanFunctions) + internal override bool IterateKeyVersions(TsavoriteKV> store, + ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) { - using SpanByteScanIterator iter = new(store, this, beginAddress, epoch, logger: logger); - return IterateKeyVersionsImpl(store, ref key, beginAddress, ref scanFunctions, iter); + using RecordScanIterator> 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) + internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver observer) { - using var iter = new SpanByteScanIterator(store: null, this, beginAddress, endAddress, ScanBufferingMode.NoBuffering, false, epoch, true, logger: logger); + using var iter = new RecordScanIterator>(store: null, this, beginAddress, endAddress, epoch, DiskScanBufferingMode.NoBuffering, InMemoryScanBufferingMode.NoBuffering, + includeSealedRecords: false, assumeInMemory: true, logger: logger); observer?.OnNext(iter); } - - /// - /// Read pages from specified device - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - internal void AsyncReadPagesFromDeviceToFrame( - long readPageStart, - int numPages, - long untilAddress, - DeviceIOCompletionCallback callback, - TContext context, - BlittableFrame frame, - out CountdownEvent completed, - long devicePageOffset = 0, - IDevice device = null, IDevice objectLogDevice = null) - { - var usedDevice = device; - if (device == null) - { - usedDevice = this.device; - } - - completed = new CountdownEvent(numPages); - for (long readPage = readPageStart; readPage < (readPageStart + numPages); readPage++) - { - int pageIndex = (int)(readPage % frame.frameSize); - if (frame.frame[pageIndex] == null) - { - frame.Allocate(pageIndex); - } - else - { - frame.Clear(pageIndex); - } - var asyncResult = new PageAsyncReadResult() - { - page = readPage, - context = context, - handle = completed, - frame = frame - }; - - ulong offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); - - uint readLength = (uint)AlignedPageSizeBytes; - long adjustedUntilAddress = (AlignedPageSizeBytes * (untilAddress >> LogPageSizeBits) + (untilAddress & PageSizeMask)); - - if (adjustedUntilAddress > 0 && ((adjustedUntilAddress - (long)offsetInFile) < PageSize)) - { - readLength = (uint)(adjustedUntilAddress - (long)offsetInFile); - readLength = (uint)((readLength + (sectorSize - 1)) & ~(sectorSize - 1)); - } - - if (device != null) - offsetInFile = (ulong)(AlignedPageSizeBytes * (readPage - devicePageOffset)); - - usedDevice.ReadAsync(offsetInFile, (IntPtr)frame.pointers[pageIndex], readLength, callback, asyncResult); - } - } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs deleted file mode 100644 index 3d75a0351af..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/SpanByteScanIterator.cs +++ /dev/null @@ -1,330 +0,0 @@ -// 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 - /// - public sealed class SpanByteScanIterator : ScanIteratorBase, ITsavoriteScanIterator, IPushScanIterator - where TStoreFunctions : IStoreFunctions - { - private readonly TsavoriteKV> store; - private readonly SpanByteAllocatorImpl hlog; - private readonly BlittableFrame frame; - - private SectorAlignedMemory memory; - private readonly bool forceInMemory; - - private long currentPhysicalAddress; - - /// - /// 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 SpanByteScanIterator(TsavoriteKV> store, SpanByteAllocatorImpl hlog, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords, LightEpoch epoch, bool forceInMemory = false, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - this.forceInMemory = forceInMemory; - if (frameSize > 0) - frame = new BlittableFrame(frameSize, hlog.PageSize, hlog.GetDeviceSectorSize()); - } - - /// - /// Constructor for use with tail-to-head push iteration of the passed key's record versions - /// - internal SpanByteScanIterator(TsavoriteKV> store, SpanByteAllocatorImpl hlog, - long beginAddress, LightEpoch epoch, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, hlog.GetTailAddress(), ScanBufferingMode.SinglePageBuffering, false, epoch, hlog.LogPageSizeBits, logger: logger) - { - this.store = store; - this.hlog = hlog; - forceInMemory = false; - if (frameSize > 0) - frame = new BlittableFrame(frameSize, hlog.PageSize, hlog.GetDeviceSectorSize()); - } - - /// - /// Gets reference to current key - /// - public ref SpanByte GetKey() => ref hlog._wrapper.GetKey(currentPhysicalAddress); - - /// - /// Gets reference to current value - /// - public ref SpanByte GetValue() => ref hlog.GetValue(currentPhysicalAddress); - - /// - 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 (!InitializeGetNext(out long headAddress, out long currentPage)) - return false; - epoch?.Suspend(); - - beginAddress = nextAddress = SnapToLogicalAddressBoundary(ref cursor, headAddress, currentPage); - return true; - } - - private bool InitializeGetNext(out long headAddress, out long currentPage) - { - currentAddress = nextAddress; - var stopAddress = endAddress < hlog.GetTailAddress() ? endAddress : hlog.GetTailAddress(); - if (currentAddress >= stopAddress) - { - headAddress = currentPage = 0; - return false; - } - - epoch?.Resume(); - headAddress = hlog.HeadAddress; - - if (currentAddress < hlog.BeginAddress && !forceInMemory) - currentAddress = hlog.BeginAddress; - - // If currentAddress < headAddress and we're not buffering and not guaranteeing the records are in memory, fail. - if (frameSize == 0 && currentAddress < headAddress && !forceInMemory) - { - epoch?.Suspend(); - throw new TsavoriteException("Iterator address is less than log HeadAddress in memory-scan mode"); - } - - currentPage = currentAddress >> hlog.LogPageSizeBits; - if (currentAddress < headAddress && !forceInMemory) - _ = 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) - { - long offset = logicalAddress & hlog.PageSizeMask; - long physicalAddress = GetPhysicalAddress(logicalAddress, headAddress, currentPage, offset) - offset; - long totalSizes = 0; - if (currentPage == 0) - { - if (logicalAddress < hlog.BeginAddress) - return logicalAddress = hlog.BeginAddress; - physicalAddress += hlog.BeginAddress; - totalSizes = (int)hlog.BeginAddress; - } - - while (totalSizes <= offset) - { - var (_, allocatedSize) = hlog.GetRecordSize(physicalAddress); - if (totalSizes + allocatedSize > offset) - break; - totalSizes += allocatedSize; - physicalAddress += allocatedSize; - } - - return logicalAddress += totalSizes - offset; - } - - /// - /// Get next record in iterator - /// - /// True if record found, false if end of scan - public unsafe bool GetNext(out RecordInfo recordInfo) - { - recordInfo = default; - - while (true) - { - if (!InitializeGetNext(out long headAddress, out long currentPage)) - return false; - - var offset = currentAddress & hlog.PageSizeMask; - long physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); - int recordSize = hlog.GetRecordSize(physicalAddress).Item2; - - // If record does not fit on page, skip to the next page. - if ((currentAddress & hlog.PageSizeMask) + recordSize > hlog.PageSize) - { - nextAddress = (1 + (currentAddress >> hlog.LogPageSizeBits)) << hlog.LogPageSizeBits; - epoch?.Suspend(); - continue; - } - - nextAddress = currentAddress + recordSize; - - recordInfo = hlog._wrapper.GetInfo(physicalAddress); - bool skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (skipOnScan || recordInfo.IsNull()) - { - epoch?.Suspend(); - continue; - } - - currentPhysicalAddress = physicalAddress; - - // 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. - if (currentAddress >= headAddress || forceInMemory) - { - OperationStackContext> stackCtx = default; - try - { - if (memory == null) - { - memory = hlog.bufferPool.Get(recordSize); - } - else - { - if (memory.AlignedTotalCapacity < recordSize) - { - memory.Return(); - memory = hlog.bufferPool.Get(recordSize); - } - } - - // GetKey() should work but for safety and consistency with other allocators use physicalAddress. - if (currentAddress >= headAddress && store is not null) - store.LockForScan(ref stackCtx, ref hlog._wrapper.GetKey(physicalAddress)); - - unsafe - { - Buffer.MemoryCopy((byte*)currentPhysicalAddress, memory.aligned_pointer, recordSize, recordSize); - currentPhysicalAddress = (long)memory.aligned_pointer; - } - } - finally - { - if (stackCtx.recSrc.HasLock) - store.UnlockForScan(ref stackCtx); - } - } - - // 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(ref SpanByte key, out RecordInfo recordInfo, out bool continueOnDisk) - { - recordInfo = default; - continueOnDisk = false; - - while (true) - { - // "nextAddress" is reused as "previous address" for this operation. - currentAddress = nextAddress; - if (currentAddress < hlog.HeadAddress) - { - continueOnDisk = currentAddress >= hlog.BeginAddress; - return false; - } - - epoch?.Resume(); - var headAddress = hlog.HeadAddress; - - var currentPage = currentAddress >> hlog.LogPageSizeBits; - var offset = currentAddress & hlog.PageSizeMask; - - long physicalAddress = GetPhysicalAddress(currentAddress, headAddress, currentPage, offset); - - recordInfo = hlog._wrapper.GetInfo(physicalAddress); - nextAddress = recordInfo.PreviousAddress; - bool skipOnScan = includeSealedRecords ? recordInfo.Invalid : recordInfo.SkipOnScan; - if (skipOnScan || recordInfo.IsNull() || !hlog._storeFunctions.KeysEqual(ref hlog._wrapper.GetKey(physicalAddress), ref key)) - { - epoch?.Suspend(); - continue; - } - - // Success; defer epoch?.Suspend(); to EndGet - currentPhysicalAddress = physicalAddress; - return true; - } - } - - bool IPushScanIterator.EndGetPrevInMemory() - { - epoch?.Suspend(); - return true; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - long GetPhysicalAddress(long currentAddress, long headAddress, long currentPage, long offset) - { - long physicalAddress; - if (currentAddress >= headAddress || forceInMemory) - physicalAddress = hlog.GetPhysicalAddress(currentAddress); - else - physicalAddress = frame.GetPhysicalAddress(currentPage % frameSize, offset); - return physicalAddress; - } - - /// - /// Get next record in iterator - /// - /// - public bool GetNext(out RecordInfo recordInfo, out SpanByte key, out SpanByte value) - => throw new NotSupportedException("Use GetNext(out RecordInfo) to retrieve references to key/value"); - - /// - /// Dispose iterator - /// - public override void Dispose() - { - base.Dispose(); - memory?.Return(); - memory = null; - frame?.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) - => hlog.AsyncReadPagesFromDeviceToFrame(readPageStart, numPages, untilAddress, AsyncReadPagesCallback, context, frame, out completed, devicePageOffset, device, objectLogDevice); - - private unsafe void AsyncReadPagesCallback(uint errorCode, uint numBytes, object context) - { - var result = (PageAsyncReadResult)context; - - if (errorCode != 0) - { - logger?.LogError($"{nameof(AsyncReadPagesCallback)} error: {{errorCode}}", errorCode); - result.cts?.Cancel(); - } - - if (result.freeBuffer1 != null) - { - hlog.PopulatePage(result.freeBuffer1.GetValidPointer(), result.freeBuffer1.required_bytes, result.page); - result.freeBuffer1.Return(); - result.freeBuffer1 = null; - } - - if (errorCode == 0) - result.handle?.Signal(); - - Interlocked.MemoryBarrier(); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs new file mode 100644 index 00000000000..35cc225f453 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocator.cs @@ -0,0 +1,149 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.CompilerServices; + +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 + using TsavoriteLogStoreFunctions = StoreFunctions; + + /// + /// Struct wrapper (for inlining) around the TsavoriteLogAllocator used by TsavoriteLog. + /// + public struct TsavoriteLogAllocator : IAllocator + { + /// The wrapped class containing all data and most actual functionality. This must be the ONLY field in this structure so its size is sizeof(IntPtr). + private readonly TsavoriteLogAllocatorImpl _this; + + public TsavoriteLogAllocator(object @this) + { + // Called by AllocatorBase via primary ctor wrapperCreator + _this = (TsavoriteLogAllocatorImpl)@this; + } + + /// + public readonly AllocatorBase GetBase() + where TAllocator : IAllocator + => (AllocatorBase)(object)_this; + + /// + public readonly bool IsFixedLength => true; + + /// + public readonly bool HasObjectLog => false; + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetStartLogicalAddress(long page) => _this.GetStartLogicalAddress(page); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetFirstValidLogicalAddress(long page) => _this.GetFirstValidLogicalAddress(page); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly long GetPhysicalAddress(long logicalAddress) => _this.GetPhysicalAddress(logicalAddress); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly void InitializeValue(long physicalAddress, ref RecordSizeInfo _) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetRMWCopyRecordSize(ref TSourceLogRecord srcLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetRMWInitialRecordSize(ReadOnlySpan key, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ReadOnlySpan value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, IHeapObject value, ref TInput input, TVariableLengthInput varlenInput) + where TVariableLengthInput : IVariableLengthInput + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetUpsertRecordSize(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordSizeInfo GetDeleteRecordSize(ReadOnlySpan key) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly void PopulateRecordSizeInfo(ref RecordSizeInfo sizeInfo) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly void AllocatePage(int pageIndex) => _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); + + /// + [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"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly LogRecord CreateLogRecord(long logicalAddress, long physicalAddress) => 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"); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) => throw new NotImplementedException("Not implemented for TsavoriteLogAllocator"); + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocatorImpl.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs similarity index 54% rename from libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocatorImpl.cs rename to libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs index fdca149f000..1f64032f1ca 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/BlittableAllocatorImpl.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/TsavoriteLogAllocatorImpl.cs @@ -8,27 +8,23 @@ namespace Tsavorite.core { - internal sealed unsafe class BlittableAllocatorImpl : AllocatorBase> - where TStoreFunctions : IStoreFunctions + // 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 static int KeySize => Unsafe.SizeOf(); - private static int ValueSize => Unsafe.SizeOf(); - internal static int RecordSize => Unsafe.SizeOf>(); - - private readonly OverflowPool overflowPagePool; + private readonly OverflowPool> overflowPagePool; - public BlittableAllocatorImpl(AllocatorSettings settings, TStoreFunctions storeFunctions, Func> wrapperCreator) - : base(settings.LogSettings, storeFunctions, wrapperCreator, settings.evictCallback, settings.epoch, settings.flushCallback, settings.logger) + public TsavoriteLogAllocatorImpl(AllocatorSettings settings) + : base(settings.LogSettings, new TsavoriteLogStoreFunctions(), @this => new TsavoriteLogAllocator(@this), evictCallback: null, settings.epoch, settings.flushCallback, settings.logger) { - if (!Utility.IsBlittable() || !Utility.IsBlittable()) - throw new TsavoriteException($"BlittableAllocator requires blittlable Key ({typeof(TKey)}) and Value ({typeof(TValue)})"); - - overflowPagePool = new OverflowPool(4, p => { }); + overflowPagePool = new OverflowPool>(4, p => { }); if (BufferSize > 0) { @@ -54,7 +50,7 @@ void ReturnPage(int index) Debug.Assert(index < BufferSize); if (values[index] != null) { - _ = overflowPagePool.TryAdd(new PageUnit + _ = overflowPagePool.TryAdd(new PageUnit { pointer = pointers[index], value = values[index] @@ -67,55 +63,6 @@ void ReturnPage(int index) public override void Initialize() => Initialize(Constants.kFirstValidAddress); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref RecordInfo GetInfo(long physicalAddress) => ref Unsafe.AsRef((void*)physicalAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref RecordInfo GetInfoFromBytePointer(byte* ptr) => ref Unsafe.AsRef(ptr); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref TKey GetKey(long physicalAddress) => ref Unsafe.AsRef((byte*)physicalAddress + RecordInfo.GetLength()); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ref TValue GetValue(long physicalAddress) => ref Unsafe.AsRef((byte*)physicalAddress + RecordInfo.GetLength() + KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static (int actualSize, int allocatedSize) GetRecordSize(long physicalAddress) => (RecordSize, RecordSize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static (int actualSize, int allocatedSize, int keySize) GetRMWCopyDestinationRecordSize(ref TKey key, ref TInput input, ref TValue value, ref RecordInfo recordInfo, TVariableLengthInput varlenInput) - => (RecordSize, RecordSize, KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static (int actualSize, int allocatedSize, int keySize) GetTombstoneRecordSize(ref TKey key) - => (RecordSize, RecordSize, KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static (int actualSize, int allocatedSize, int keySize) GetRMWInitialRecordSize(ref TKey key, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - => (RecordSize, RecordSize, KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int GetRequiredRecordSize(long physicalAddress, int availableBytes) => GetAverageRecordSize(); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int GetAverageRecordSize() => RecordSize; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static int GetFixedRecordSize() => RecordSize; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static (int actualSize, int allocatedSize, int keySize) GetRecordSize(ref TKey key, ref TValue value) => (RecordSize, RecordSize, KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static (int actualSize, int allocatedSize, int keySize) GetUpsertRecordSize(ref TKey key, ref TValue value, ref TInput input, TSessionFunctionsWrapper sessionFunctions) - => (RecordSize, RecordSize, KeySize); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int GetValueLength(ref TValue value) => ValueSize; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static void SerializeKey(ref TKey src, long physicalAddress) => GetKey(physicalAddress) = src; - /// /// Dispose memory allocator /// @@ -185,21 +132,6 @@ protected override void WriteAsyncToDevice device); } - /// - /// Get start logical address - /// - public long GetStartLogicalAddress(long page) => page << LogPageSizeBits; - - /// - /// Get first valid logical address - /// - public long GetFirstValidLogicalAddress(long page) - { - if (page == 0) - return (page << LogPageSizeBits) + Constants.kFirstValidAddress; - return page << LogPageSizeBits; - } - internal void ClearPage(long page, int offset) { if (offset == 0) @@ -219,15 +151,6 @@ internal void FreePage(long page) ReturnPage((int)(page % BufferSize)); } - /// - /// Delete in-memory portion of the log - /// - internal override void DeleteFromMemory() - { - for (int i = 0; i < values.Length; i++) - values[i] = null; - } - 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); @@ -241,63 +164,42 @@ protected override void ReadAsync(ulong alignedSourceAddress, int dest /// /// /// - protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) - => throw new InvalidOperationException("AsyncReadRecordObjectsToMemory invalid for BlittableAllocator"); - - internal static bool RetrievedFullRecord(byte* record, ref AsyncIOContext ctx) - { - ctx.key = GetKey((long)record); - ctx.value = GetValue((long)record); - return true; - } - - internal static long[] GetSegmentOffsets() => null; + protected override void AsyncReadRecordObjectsToMemory(long fromLogical, int numBytes, DeviceIOCompletionCallback callback, AsyncIOContext context, SectorAlignedMemory result = default) + => throw new InvalidOperationException("AsyncReadRecordObjectsToMemory invalid for TsavoriteLogAllocator"); internal static void PopulatePage(byte* src, int required_bytes, long destinationPage) - => throw new TsavoriteException("BlittableAllocator memory pages are sector aligned - use direct copy"); + => throw new TsavoriteException("TsavoriteLogAllocator memory pages are sector aligned - use direct copy"); /// /// Iterator interface for pull-scanning Tsavorite log /// - public override ITsavoriteScanIterator Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode, bool includeSealedRecords) - => new BlittableScanIterator(store, this, beginAddress, endAddress, scanBufferingMode, includeSealedRecords, epoch, logger: logger); + public override ITsavoriteScanIterator Scan(TsavoriteKV store, + long beginAddress, long endAddress, DiskScanBufferingMode diskScanBufferingMode, bool includeSealedRecords) + => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// /// Implementation for push-scanning Tsavorite log, called from LogAccessor /// - internal override bool Scan(TsavoriteKV> store, - long beginAddress, long endAddress, ref TScanFunctions scanFunctions, ScanBufferingMode scanBufferingMode) - { - using BlittableScanIterator iter = new(store, this, beginAddress, endAddress, scanBufferingMode, false, epoch, logger: logger); - return PushScanImpl(beginAddress, endAddress, ref scanFunctions, iter); - } + internal override bool Scan(TsavoriteKV store, + long beginAddress, long endAddress, ref TScanFunctions scanFunctions, DiskScanBufferingMode diskScanBufferingMode) + => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// /// Implementation for push-scanning Tsavorite log with a cursor, called from LogAccessor /// - internal override bool ScanCursor(TsavoriteKV> store, - ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) - { - using BlittableScanIterator iter = new(store, this, cursor, endAddress, ScanBufferingMode.SinglePageBuffering, false, epoch, logger: logger); - return ScanLookup>(store, scanCursorState, ref cursor, count, scanFunctions, iter, validateCursor, maxAddress); - } + internal override bool ScanCursor(TsavoriteKV store, + ScanCursorState scanCursorState, ref long cursor, long count, TScanFunctions scanFunctions, long endAddress, bool validateCursor, long maxAddress) + => 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, ref TKey key, long beginAddress, ref TScanFunctions scanFunctions) - { - using BlittableScanIterator iter = new(store, this, beginAddress, epoch, logger: logger); - return IterateKeyVersionsImpl(store, ref key, beginAddress, ref scanFunctions, iter); - } + internal override bool IterateKeyVersions(TsavoriteKV store, ReadOnlySpan key, long beginAddress, ref TScanFunctions scanFunctions) + => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// - internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver> observer) - { - using var iter = new BlittableScanIterator(store: null, this, beginAddress, endAddress, ScanBufferingMode.NoBuffering, false, epoch, true, logger: logger); - observer?.OnNext(iter); - } + internal override void MemoryPageScan(long beginAddress, long endAddress, IObserver observer) + => throw new TsavoriteException("TsavoriteLogAllocator Scan methods should not be used"); /// /// Read pages from specified device diff --git a/libs/storage/Tsavorite/cs/src/core/Async/CompletePendingAsync.cs b/libs/storage/Tsavorite/cs/src/core/Async/CompletePendingAsync.cs index a4c47e14fff..4d9555ad554 100644 --- a/libs/storage/Tsavorite/cs/src/core/Async/CompletePendingAsync.cs +++ b/libs/storage/Tsavorite/cs/src/core/Async/CompletePendingAsync.cs @@ -9,9 +9,9 @@ namespace Tsavorite.core /// /// The Tsavorite key-value store /// - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Check if at least one (sync) request is ready for CompletePending to operate on @@ -28,8 +28,8 @@ internal static ValueTask ReadyToCompletePendingAsync /// /// internal async ValueTask CompletePendingAsync(TSessionFunctionsWrapper sessionFunctions, - CancellationToken token, CompletedOutputIterator completedOutputs) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + CancellationToken token, CompletedOutputIterator completedOutputs) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { while (true) { @@ -44,11 +44,10 @@ internal async ValueTask CompletePendingAsync(sessionFunctions); - Thread.Yield(); } } diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/BasicContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/BasicContext.cs index 1f791debe10..c3413b0c1ac 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/BasicContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/BasicContext.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Runtime.CompilerServices; using System.Threading; using System.Threading.Tasks; @@ -10,21 +11,21 @@ namespace Tsavorite.core /// /// Basic Tsavorite Context implementation. /// - public readonly struct BasicContext - : ITsavoriteContext - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public readonly struct BasicContext + : ITsavoriteContext + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - readonly ClientSession clientSession; - internal readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; + readonly ClientSession clientSession; + internal readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; /// public bool IsNull => clientSession is null; - private TsavoriteKV store => clientSession.store; + private TsavoriteKV store => clientSession.store; - internal BasicContext(ClientSession clientSession) + internal BasicContext(ClientSession clientSession) { this.clientSession = clientSession; sessionFunctions = new(clientSession); @@ -43,20 +44,17 @@ public void UnsafeSuspendThread() #region ITsavoriteContext /// - public ClientSession Session => clientSession; + public ClientSession Session => clientSession; /// - public long GetKeyHash(TKey key) => clientSession.store.GetKeyHash(ref key); - - /// - public long GetKeyHash(ref TKey key) => clientSession.store.GetKeyHash(ref key); + public long GetKeyHash(ReadOnlySpan key) => clientSession.store.GetKeyHash(key); /// public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) => clientSession.CompletePending(sessionFunctions, wait, spinWaitForCommit); /// - public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) => clientSession.CompletePendingWithOutputs(sessionFunctions, out completedOutputs, wait, spinWaitForCommit); /// @@ -64,17 +62,17 @@ public ValueTask CompletePendingAsync(bool waitForCommit = false, CancellationTo => clientSession.CompletePendingAsync(sessionFunctions, waitForCommit, token); /// - public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) + public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) => clientSession.CompletePendingWithOutputsAsync(sessionFunctions, waitForCommit, token); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) { UnsafeResumeThread(); try { - return clientSession.store.ContextRead(ref key, ref input, ref output, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, userContext, sessionFunctions); } finally { @@ -84,85 +82,51 @@ public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - => Read(ref key, ref input, ref output, ref readOptions, out _, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + => Read(key, ref input, ref output, ref readOptions, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, TContext userContext = default) { TInput input = default; - output = default; - return Read(ref key, ref input, ref output, userContext); + return Read(key, ref input, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); + return Read(key, ref input, ref output, ref readOptions, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, userContext), output); + return (Read(key, ref input, ref output, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, ref ReadOptions readOptions, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, ref readOptions, userContext), output); + return (Read(key, ref input, ref output, ref readOptions, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { UnsafeResumeThread(); try { - return store.ContextRead(ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return store.ContextRead(key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } finally { @@ -187,12 +151,12 @@ public Status ReadAtAddress(long address, ref TInput input, ref TOutput output, /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status ReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { UnsafeResumeThread(); try { - return store.ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return store.ContextReadAtAddress(address, key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } finally { @@ -202,40 +166,40 @@ public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TO /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + return Upsert(key, store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + return Upsert(key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) { UnsafeResumeThread(); try { - return store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, userContext, sessionFunctions); + return store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out _, userContext, sessionFunctions); } finally { @@ -245,22 +209,13 @@ private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue d /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) { + var keyHash = upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key); UnsafeResumeThread(); try { - return store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + return store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); } finally { @@ -270,53 +225,56 @@ private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue d /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, TContext userContext = default) - => Upsert(ref key, ref desiredValue, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref desiredValue, ref upsertOptions, userContext); + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, ref upsertOptions, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) - => RMW(ref key, store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out _, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out _, userContext); - - /// - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); - - /// - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) { - var keyHash = rmwOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key); - return RMW(ref key, keyHash, ref input, ref output, out recordMetadata, userContext); + UnsafeResumeThread(); + try + { + return store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out _, userContext, sessionFunctions); + } + finally + { + UnsafeSuspendThread(); + } } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) { + var keyHash = upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key); UnsafeResumeThread(); try { - return store.ContextRMW(ref key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); + return store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); } finally { @@ -326,64 +284,74 @@ private Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput out /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status Upsert(ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + => Upsert(diskLogRecord.Key, ref diskLogRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord { - output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + TInput input = default; + TOutput output = default; + UpsertOptions upsertOptions = default; + return Upsert(key, ref input, ref diskLogRecord, ref output, ref upsertOptions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status Upsert(ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord + => Upsert(inputLogRecord.Key, ref input, ref inputLogRecord, ref output, ref upsertOptions, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord { - TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + var keyHash = upsertOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key); + + UnsafeResumeThread(); + try + { + return store.ContextUpsert(key, keyHash, ref input, inputLogRecord: ref inputLogRecord, ref output, out _, userContext, sessionFunctions); + } + finally + { + UnsafeSuspendThread(); + } } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, TContext userContext = default) - => RMW(ref key, ref input, userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) + => RMW(key, store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, ref RMWOptions rmwOptions, TContext userContext = default) - => RMW(ref key, ref input, ref rmwOptions, userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out _, userContext); /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, TContext userContext = default) - => Delete(ref key, store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, deleteOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + var keyHash = rmwOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key); + return RMW(key, keyHash, ref input, ref output, out recordMetadata, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Delete(ref TKey key, long keyHash, TContext userContext = default) + private Status RMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) { UnsafeResumeThread(); try { - return store.ContextDelete, TStoreFunctions, TAllocator>>(ref key, keyHash, userContext, sessionFunctions); + return store.ContextRMW(key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); } finally { @@ -393,52 +361,74 @@ private Status Delete(ref TKey key, long keyHash, TContext userContext = default /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, TContext userContext = default) - => Delete(ref key, userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, TContext userContext = default) + { + TOutput output = default; + return RMW(key, ref input, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + { + TOutput output = default; + return RMW(key, ref input, ref output, ref rmwOptions, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Delete(ReadOnlySpan key, TContext userContext = default) + => Delete(key, store.storeFunctions.GetKeyHashCode64(key), userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, ref deleteOptions, userContext); + public Status Delete(ReadOnlySpan key, ref DeleteOptions deleteOptions, TContext userContext = default) + => Delete(key, deleteOptions.KeyHash ?? store.storeFunctions.GetKeyHashCode64(key), userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ResetModified(TKey key) - => clientSession.ResetModified(sessionFunctions, ref key); + private Status Delete(ReadOnlySpan key, long keyHash, TContext userContext = default) + { + UnsafeResumeThread(); + try + { + return store.ContextDelete, TStoreFunctions, TAllocator>>(key, keyHash, userContext, sessionFunctions); + } + finally + { + UnsafeSuspendThread(); + } + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ResetModified(ref TKey key) - => clientSession.ResetModified(sessionFunctions, ref key); + public void ResetModified(ReadOnlySpan key) => clientSession.ResetModified(sessionFunctions, key); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool IsModified(TKey key) - => clientSession.IsModified(sessionFunctions, ref key); + internal bool IsModified(ReadOnlySpan key) => clientSession.IsModified(sessionFunctions, key); /// - public void Refresh() - => clientSession.Refresh(sessionFunctions); + public void Refresh() => clientSession.Refresh(sessionFunctions); #endregion ITsavoriteContext /// /// Copy key and value to tail, succeed only if key is known to not exist in between expectedLogicalAddress and tail. /// - /// - /// - /// - /// + /// /// LogicalAddress of the record to be copied /// Lower-bound address (addresses are searched from tail (high) to head (low); do not search for "future records" earlier than this) [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status CompactionCopyToTail(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, long currentAddress, long untilAddress) + internal Status CompactionCopyToTail(ref TSourceLogRecord srcLogRecord, long currentAddress, long untilAddress) + where TSourceLogRecord : ISourceLogRecord { UnsafeResumeThread(); try { - return store.CompactionConditionalCopyToTail, TStoreFunctions, TAllocator>>( - sessionFunctions, ref key, ref input, ref value, ref output, currentAddress, untilAddress); + return store.CompactionConditionalCopyToTail, TStoreFunctions, TAllocator>, TSourceLogRecord>( + sessionFunctions, ref srcLogRecord, currentAddress, untilAddress); } finally { @@ -450,19 +440,19 @@ internal Status CompactionCopyToTail(ref TKey key, ref TInput input, ref TValue /// Push a scan record to client if key is known to not exist in between expectedLogicalAddress and tail. /// /// Scan cursor tracking state, from the session on which this scan was initiated - /// - /// - /// + /// /// LogicalAddress of the record to be copied /// Lower-bound address (addresses are searched from tail (high) to head (low); do not search for "future records" earlier than this) + /// Maximum address for determining liveness, records after this address are not considered when checking validity. [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ConditionalScanPush(ScanCursorState scanCursorState, RecordInfo recordInfo, ref TKey key, ref TValue value, long currentAddress, long untilAddress, long maxAddress) + internal Status ConditionalScanPush(ScanCursorState scanCursorState, ref TSourceLogRecord srcLogRecord, long currentAddress, long untilAddress, long maxAddress) + where TSourceLogRecord : ISourceLogRecord { UnsafeResumeThread(); try { - return store.hlogBase.ConditionalScanPush, TStoreFunctions, TAllocator>>( - sessionFunctions, scanCursorState, recordInfo, ref key, ref value, currentAddress, untilAddress, maxAddress); + return store.hlogBase.ConditionalScanPush, TStoreFunctions, TAllocator>, TSourceLogRecord>( + sessionFunctions, scanCursorState, ref srcLogRecord, currentAddress, untilAddress, maxAddress); } finally { @@ -477,13 +467,13 @@ internal Status ConditionalScanPush(ScanCursorState scanCursorStat /// Logical address of record, if found /// Look until this address; if less than HeadAddress, then HeadAddress is used /// Status - internal Status ContainsKeyInMemory(ref TKey key, out long logicalAddress, long fromAddress = -1) + internal Status ContainsKeyInMemory(ReadOnlySpan key, out long logicalAddress, long fromAddress = -1) // TODO: remove when we remove tempkv/tempdb in iterators { UnsafeResumeThread(); try { - return store.InternalContainsKeyInMemory, TStoreFunctions, TAllocator>>( - ref key, sessionFunctions, out logicalAddress, fromAddress); + return store.InternalContainsKeyInMemory, TStoreFunctions, TAllocator>>( + key, sessionFunctions, out logicalAddress, fromAddress); } finally { diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 580c5e7a507..4bee9ce4f4a 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -13,23 +13,23 @@ namespace Tsavorite.core /// /// Thread-independent session interface to Tsavorite /// - public sealed class ClientSession : IClientSession, IDisposable - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public sealed class ClientSession : IClientSession, IDisposable + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - internal readonly TsavoriteKV store; + internal readonly TsavoriteKV store; - internal readonly TsavoriteKV.TsavoriteExecutionContext ctx; + internal readonly TsavoriteKV.TsavoriteExecutionContext ctx; internal readonly TFunctions functions; - internal CompletedOutputIterator completedOutputs; + internal CompletedOutputIterator completedOutputs; - readonly UnsafeContext uContext; - readonly LockableUnsafeContext luContext; - readonly LockableContext lContext; - readonly BasicContext bContext; + readonly UnsafeContext uContext; + readonly TransactionalUnsafeContext luContext; + readonly TransactionalContext lContext; + readonly BasicContext bContext; internal const string NotAsyncSessionErr = "Session does not support async operations"; @@ -40,49 +40,49 @@ public sealed class ClientSession scanCursorState; + ScanCursorState scanCursorState; - internal void AcquireLockable(TSessionFunctions sessionFunctions) - where TSessionFunctions : ISessionFunctionsWrapper + internal void AcquireTransactional(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - CheckIsNotAcquiredLockable(sessionFunctions); - sessionFunctions.Ctx.isAcquiredLockable = true; + CheckIsNotAcquiredTransactional(sessionFunctions); + sessionFunctions.Ctx.isAcquiredTransactional = true; } internal void LocksAcquired(TSessionFunctions sessionFunctions, long txnVersion) - where TSessionFunctions : ISessionFunctionsWrapper + where TSessionFunctions : ISessionFunctionsWrapper { - CheckIsAcquiredLockable(sessionFunctions); + CheckIsAcquiredTransactional(sessionFunctions); sessionFunctions.Ctx.txnVersion = txnVersion; } - internal void ReleaseLockable(TSessionFunctions sessionFunctions) - where TSessionFunctions : ISessionFunctionsWrapper + internal void ReleaseTransactional(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - CheckIsAcquiredLockable(sessionFunctions); + CheckIsAcquiredTransactional(sessionFunctions); if (TotalLockCount > 0) - throw new TsavoriteException($"EndLockable called with locks held: {sharedLockCount} shared locks, {exclusiveLockCount} exclusive locks"); - sessionFunctions.Ctx.isAcquiredLockable = false; + throw new TsavoriteException($"EndTransactional called with locks held: {sharedLockCount} shared locks, {exclusiveLockCount} exclusive locks"); + sessionFunctions.Ctx.isAcquiredTransactional = false; sessionFunctions.Ctx.txnVersion = 0; } - internal void CheckIsAcquiredLockable(TSessionFunctions sessionFunctions) - where TSessionFunctions : ISessionFunctionsWrapper + internal void CheckIsAcquiredTransactional(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - if (!sessionFunctions.Ctx.isAcquiredLockable) - throw new TsavoriteException("Lockable method call when BeginLockable has not been called"); + if (!sessionFunctions.Ctx.isAcquiredTransactional) + throw new TsavoriteException("Transactional method call when BeginTransactional has not been called"); } - void CheckIsNotAcquiredLockable(TSessionFunctions sessionFunctions) - where TSessionFunctions : ISessionFunctionsWrapper + void CheckIsNotAcquiredTransactional(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - if (sessionFunctions.Ctx.isAcquiredLockable) - throw new TsavoriteException("BeginLockable cannot be called twice (call EndLockable first)"); + if (sessionFunctions.Ctx.isAcquiredTransactional) + throw new TsavoriteException("BeginTransactional cannot be called twice (call EndTransactional first)"); } internal ClientSession( - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, + TsavoriteKV store, + TsavoriteKV.TsavoriteExecutionContext ctx, TFunctions functions, ILoggerFactory loggerFactory = null) { @@ -125,34 +125,34 @@ public void Dispose() /// /// Return a new interface to Tsavorite operations that supports manual epoch control. /// - public UnsafeContext UnsafeContext => uContext; + public UnsafeContext UnsafeContext => uContext; /// - /// Return a new interface to Tsavorite operations that supports manual locking and epoch control. + /// Return a new interface to Tsavorite operations that supports Transactional locking and manual epoch control. /// - public LockableUnsafeContext LockableUnsafeContext => luContext; + public TransactionalUnsafeContext TransactionalUnsafeContext => luContext; /// - /// Return a session wrapper that supports manual locking. + /// Return a session wrapper that supports Transactional locking. /// - public LockableContext LockableContext => lContext; + public TransactionalContext TransactionalContext => lContext; /// /// Return a session wrapper struct that passes through to client session /// - public BasicContext BasicContext => bContext; + public BasicContext BasicContext => bContext; #region ITsavoriteContext /// - public long GetKeyHash(TKey key) => store.GetKeyHash(ref key); + public long GetKeyHash(ReadOnlySpan key) => store.GetKeyHash(key); /// - public long GetKeyHash(ref TKey key) => store.GetKeyHash(ref key); + public long GetKeyHash(ref ReadOnlySpan key) => store.GetKeyHash(key); /// internal void Refresh(TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { UnsafeResumeThread(sessionFunctions); try @@ -166,13 +166,13 @@ internal void Refresh(TSessionFunctionsWrapper session } /// - internal void ResetModified(TSessionFunctionsWrapper sessionFunctions, ref TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal void ResetModified(TSessionFunctionsWrapper sessionFunctions, ReadOnlySpan key) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { UnsafeResumeThread(sessionFunctions); try { - UnsafeResetModified(sessionFunctions, ref key); + UnsafeResetModified(sessionFunctions, key); } finally { @@ -181,16 +181,16 @@ internal void ResetModified(TSessionFunctionsWrapper s } /// - public int CompareKeyHashes(TLockableKey key1, TLockableKey key2) where TLockableKey : ILockableKey => store.LockTable.CompareKeyHashes(key1, key2); + public int CompareKeyHashes(TTransactionalKey key1, TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => store.LockTable.CompareKeyHashes(key1, key2); /// - public int CompareKeyHashes(ref TLockableKey key1, ref TLockableKey key2) where TLockableKey : ILockableKey => store.LockTable.CompareKeyHashes(ref key1, ref key2); + public int CompareKeyHashes(ref TTransactionalKey key1, ref TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => store.LockTable.CompareKeyHashes(ref key1, ref key2); /// - public void SortKeyHashes(TLockableKey[] keys) where TLockableKey : ILockableKey => store.LockTable.SortKeyHashes(keys); + public void SortKeyHashes(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => store.LockTable.SortKeyHashes(keys); /// - public void SortKeyHashes(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey => store.LockTable.SortKeyHashes(keys, start, count); + public void SortKeyHashes(TTransactionalKey[] keys, int start, int count) where TTransactionalKey : ITransactionalKey => store.LockTable.SortKeyHashes(keys, start, count); #endregion ITsavoriteContext @@ -198,12 +198,12 @@ internal void ResetModified(TSessionFunctionsWrapper s /// internal bool CompletePending(TSessionFunctionsWrapper sessionFunctions, bool wait = false, bool spinWaitForCommit = false) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper => CompletePending(sessionFunctions, getOutputs: false, wait, spinWaitForCommit); /// - internal bool CompletePendingWithOutputs(TSessionFunctionsWrapper sessionFunctions, out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal bool CompletePendingWithOutputs(TSessionFunctionsWrapper sessionFunctions, out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { InitializeCompletedOutputs(); var result = CompletePending(sessionFunctions, getOutputs: true, wait, spinWaitForCommit); @@ -215,8 +215,8 @@ internal bool CompletePendingWithOutputs(TSessionFunct /// Synchronously complete outstanding pending synchronous operations, returning outputs for the completed operations. /// Assumes epoch protection is managed by user. Async operations must be completed individually. /// - internal bool UnsafeCompletePendingWithOutputs(TSessionFunctionsWrapper sessionFunctions, out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal bool UnsafeCompletePendingWithOutputs(TSessionFunctionsWrapper sessionFunctions, out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { InitializeCompletedOutputs(); var result = UnsafeCompletePending(sessionFunctions, true, wait, spinWaitForCommit); @@ -227,13 +227,13 @@ internal bool UnsafeCompletePendingWithOutputs(TSessio private void InitializeCompletedOutputs() { if (completedOutputs is null) - completedOutputs = new CompletedOutputIterator(); + completedOutputs = new CompletedOutputIterator(); else completedOutputs.Dispose(); } internal bool CompletePending(TSessionFunctionsWrapper sessionFunctions, bool getOutputs, bool wait, bool spinWaitForCommit) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { UnsafeResumeThread(sessionFunctions); try @@ -247,7 +247,7 @@ internal bool CompletePending(TSessionFunctionsWrapper } internal bool UnsafeCompletePending(TSessionFunctionsWrapper sessionFunctions, bool getOutputs, bool wait, bool spinWaitForCommit) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var requestedOutputs = getOutputs ? completedOutputs : default; var result = store.InternalCompletePending(sessionFunctions, wait, requestedOutputs); @@ -270,13 +270,13 @@ internal bool UnsafeCompletePending(TSessionFunctionsW /// internal ValueTask CompletePendingAsync(TSessionFunctionsWrapper sessionFunctions, bool waitForCommit = false, CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper => CompletePendingAsync(sessionFunctions, getOutputs: false, waitForCommit, token); /// - internal async ValueTask> CompletePendingWithOutputsAsync(TSessionFunctionsWrapper sessionFunctions, + internal async ValueTask> CompletePendingWithOutputsAsync(TSessionFunctionsWrapper sessionFunctions, bool waitForCommit = false, CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { InitializeCompletedOutputs(); await CompletePendingAsync(sessionFunctions, getOutputs: true, waitForCommit, token).ConfigureAwait(false); @@ -284,7 +284,7 @@ internal async ValueTask(TSessionFunctionsWrapper sessionFunctions, bool getOutputs, bool waitForCommit = false, CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { token.ThrowIfCancellationRequested(); @@ -312,35 +312,30 @@ public async ValueTask ReadyToCompletePendingAsync(CancellationToken token = def if (store.epoch.ThisInstanceProtected()) throw new NotSupportedException("Async operations not supported over protected epoch"); - await TsavoriteKV.ReadyToCompletePendingAsync(ctx, token).ConfigureAwait(false); + await TsavoriteKV.ReadyToCompletePendingAsync(ctx, token).ConfigureAwait(false); } #endregion Pending Operations #region Other Operations - internal void UnsafeResetModified(TSessionFunctionsWrapper sessionFunctions, ref TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal void UnsafeResetModified(TSessionFunctionsWrapper sessionFunctions, ReadOnlySpan key) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { OperationStatus status; do - status = store.InternalModifiedBitOperation(ref key, out _); + status = store.InternalModifiedBitOperation(key, out _); while (store.HandleImmediateNonPendingRetryStatus(status, sessionFunctions)); } /// - internal unsafe void ResetModified(TSessionFunctionsWrapper sessionFunctions, TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - => ResetModified(sessionFunctions, ref key); - - /// - internal bool IsModified(TSessionFunctionsWrapper sessionFunctions, ref TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal bool IsModified(TSessionFunctionsWrapper sessionFunctions, ReadOnlySpan key) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { UnsafeResumeThread(sessionFunctions); try { - return UnsafeIsModified(sessionFunctions, ref key); + return UnsafeIsModified(sessionFunctions, key); } finally { @@ -348,29 +343,24 @@ internal bool IsModified(TSessionFunctionsWrapper sess } } - internal bool UnsafeIsModified(TSessionFunctionsWrapper sessionFunctions, ref TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal bool UnsafeIsModified(TSessionFunctionsWrapper sessionFunctions, ReadOnlySpan key) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { RecordInfo modifiedInfo; OperationStatus status; do - status = store.InternalModifiedBitOperation(ref key, out modifiedInfo, false); + status = store.InternalModifiedBitOperation(key, out modifiedInfo, false); while (store.HandleImmediateNonPendingRetryStatus(status, sessionFunctions)); return modifiedInfo.Modified; } - /// - internal unsafe bool IsModified(TSessionFunctionsWrapper sessionFunctions, TKey key) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - => IsModified(sessionFunctions, ref key); - /// /// Wait for commit of all operations completed until the current point in session. /// Does not itself issue checkpoint/commits. /// /// private async ValueTask WaitForCommitAsync(TSessionFunctionsWrapper sessionFunctions, CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { token.ThrowIfCancellationRequested(); @@ -398,19 +388,7 @@ private async ValueTask WaitForCommitAsync(TSessionFun /// Compaction type (whether we lookup records or scan log for liveness checking) /// Address until which compaction was done public long Compact(long compactUntilAddress, CompactionType compactionType = CompactionType.Scan) - => Compact(compactUntilAddress, compactionType, default(DefaultCompactionFunctions)); - - /// - /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log - /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate - /// - /// Input for SingleWriter - /// Output from SingleWriter; it will be called all records that are moved, before Compact() returns, so the user must supply buffering or process each output completely - /// Compact log until this address - /// Compaction type (whether we lookup records or scan log for liveness checking) - /// Address until which compaction was done - public long Compact(ref TInput input, ref TOutput output, long compactUntilAddress, CompactionType compactionType = CompactionType.Scan) - => Compact(ref input, ref output, compactUntilAddress, compactionType, default(DefaultCompactionFunctions)); + => Compact(compactUntilAddress, compactionType, default(DefaultCompactionFunctions)); /// /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log @@ -418,38 +396,18 @@ public long Compact(ref TInput input, ref TOutput output, long compactUntilAddre /// /// Compact log until this address /// Compaction type (whether we lookup records or scan log for liveness checking) - /// User provided compaction functions (see ). + /// User provided compaction functions (see ). /// Address until which compaction was done public long Compact(long untilAddress, CompactionType compactionType, CompactionFunctions compactionFunctions) - where CompactionFunctions : ICompactionFunctions - { - TInput input = default; - TOutput output = default; - return store.Compact(functions, compactionFunctions, ref input, ref output, untilAddress, compactionType); - } - - /// - /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log - /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate - /// - /// Input for SingleWriter - /// Output from SingleWriter; it will be called all records that are moved, before Compact() returns, so the user must supply buffering or process each output completely - /// Compact log until this address - /// Compaction type (whether we lookup records or scan log for liveness checking) - /// User provided compaction functions (see ). - /// Address until which compaction was done - public long Compact(ref TInput input, ref TOutput output, long untilAddress, CompactionType compactionType, CompactionFunctions compactionFunctions) - where CompactionFunctions : ICompactionFunctions - { - return store.Compact(functions, compactionFunctions, ref input, ref output, untilAddress, compactionType); - } + where CompactionFunctions : ICompactionFunctions + => store.Compact(compactionFunctions, untilAddress, compactionType); /// /// Pull iterator for all (distinct) live key-values stored in Tsavorite /// /// Report records until this address (tail by default) /// Tsavorite iterator - public ITsavoriteScanIterator Iterate(long untilAddress = -1) + public ITsavoriteScanIterator Iterate(long untilAddress = -1) => store.Iterate(functions, untilAddress); /// @@ -459,7 +417,7 @@ public ITsavoriteScanIterator Iterate(long untilAddress = -1) /// Report records until this address (tail by default) /// True if Iteration completed; false if Iteration ended early due to one of the TScanIterator reader functions returning false public bool Iterate(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => store.Iterate(functions, ref scanFunctions, untilAddress); /// @@ -469,7 +427,7 @@ public bool Iterate(ref TScanFunctions scanFunctions, long until /// Report records until this address (tail by default) /// True if Iteration completed; false if Iteration ended early due to one of the TScanIterator reader functions returning false public bool IterateLookup(ref TScanFunctions scanFunctions, long untilAddress = -1) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions { if (untilAddress == -1) untilAddress = store.Log.TailAddress; @@ -496,7 +454,7 @@ public bool IterateLookup(ref TScanFunctions scanFunctions, long /// or one of the TScanIterator reader functions returning false, or if we determined that there are no records remaining. In other words, if this returns true, /// there may be more records satisfying the iteration criteria beyond . public bool ScanCursor(ref long cursor, long count, TScanFunctions scanFunctions, long endAddress = long.MaxValue, bool validateCursor = false, long maxAddress = long.MaxValue) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions => store.hlogBase.ScanCursor(store, scanCursorState ??= new(), ref cursor, count, scanFunctions, endAddress, validateCursor, maxAddress); /// @@ -504,7 +462,7 @@ public bool ScanCursor(ref long cursor, long count, TScanFunctio /// [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void UnsafeResumeThread(TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // We do not track any "acquired" state here; if someone mixes calls between safe and unsafe contexts, they will // get the "trying to acquire already-acquired epoch" error. diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs similarity index 53% rename from libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs rename to libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs index 7f50bf32fa8..b6b7b385a87 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITransactionalContext.cs @@ -7,17 +7,16 @@ namespace Tsavorite.core { /// - /// Lockable context functions. Useful when doing generic locking across diverse - /// and - /// specializations. + /// Transactional context functions. Useful when doing generic locking across diverse + /// and + /// specializations. /// - /// - public interface ILockableContext + public interface ITransactionalContext { /// /// Begins a series of lock operations on possibly multiple keys; call before any locks are taken. /// - void BeginLockable(); + void BeginTransaction(); /// /// Call after all locks are acquired. Provide transaction version @@ -28,180 +27,180 @@ public interface ILockableContext /// /// Ends a series of lock operations on possibly multiple keys; call after all locks are released. /// - void EndLockable(); + void EndTransaction(); /// - /// Compare two structures that implement ILockableKey. + /// Compare two structures that implement . /// - /// The type of the app data struct or class containing key info + /// The type of the app data struct or class containing key info /// The first key to compare /// The first key to compare /// The result of key1.CompareTo(key2) - int CompareKeyHashes(TLockableKey key1, TLockableKey key2) - where TLockableKey : ILockableKey; + int CompareKeyHashes(TTransactionalKey key1, TTransactionalKey key2) + where TTransactionalKey : ITransactionalKey; /// - /// Compare two structures that implement ILockableKey. + /// Compare two structures that implement . /// - /// The type of the app data struct or class containing key info + /// The type of the app data struct or class containing key info /// The first key to compare /// The first key to compare /// The result of key1.CompareTo(key2) - int CompareKeyHashes(ref TLockableKey key1, ref TLockableKey key2) - where TLockableKey : ILockableKey; + int CompareKeyHashes(ref TTransactionalKey key1, ref TTransactionalKey key2) + where TTransactionalKey : ITransactionalKey; /// - /// Sort an array of app data structures (or classes) by lock code and lock type; these will be passed to Lockable*Session.Lock + /// Sort an array of app data structures (or classes) by lock code and lock type; these will be passed to Transactional*Session.Lock /// - /// The type of the app data struct or class containing key info + /// The type of the app data struct or class containing key info /// The array of app key data - void SortKeyHashes(TLockableKey[] keys) - where TLockableKey : ILockableKey; + void SortKeyHashes(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey; /// - /// Sort an array of app data structures (or classes) by lock code and lock type; these will be passed to Lockable*Session.Lock + /// Sort an array of app data structures (or classes) by lock code and lock type; these will be passed to Transactional*Session.Lock /// - /// The type of the app data struct or class containing key info + /// The type of the app data struct or class containing key info /// The array of app key data /// The starting key index to sort /// The number of keys to sort - void SortKeyHashes(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey; + void SortKeyHashes(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . - void Lock(TLockableKey[] keys) - where TLockableKey : ILockableKey; + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + void Lock(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array. /// - /// - /// key hashCodes to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// key hashCodes to be locked, and whether that locking is shared or exclusive; must be sorted by . /// The starting key index to Lock /// The number of keys to Lock - void Lock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey; + void Lock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . - bool TryLock(TLockableKey[] keys) - where TLockableKey : ILockableKey; + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + bool TryLock(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . /// TimeSpan limiting the duration of the TryLock() call over all keys. - bool TryLock(TLockableKey[] keys, TimeSpan timeout) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . /// The starting key index to Lock /// The number of keys to Lock /// TimeSpan limiting the duration of the TryLock() call over all keys. - bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . /// The cancellation token - bool TryLock(TLockableKey[] keys, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . /// The starting key index to Lock /// The number of keys to Lock /// The cancellation token, if any - bool TryLock(TLockableKey[] keys, int start, int count, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, int start, int count, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Locks the keys identified in the passed array, with retry limits or cancellation. /// - /// - /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// keys to be locked, and whether that locking is shared or exclusive; must be sorted by . /// TimeSpan limiting the duration of the TryLock() call over all keys. /// The cancellation token - bool TryLock(TLockableKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Promotes a shared lock on the key to an exclusive lock, with retry limits or cancellation. /// - /// - /// key hashCodes to be locked, and whether that locking is shared or exclusive; must be sorted by . + /// + /// key hashCodes to be locked, and whether that locking is shared or exclusive; must be sorted by . /// The starting key index to Lock /// The number of keys to Lock /// TimeSpan limiting the duration of the TryLock() call over all keys. /// The cancellation token, if any - bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Tries to promote a shared lock the key to an exclusive lock, with retry limits or cancellation. /// - /// + /// /// 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(TLockableKey key) - where TLockableKey : ILockableKey; + /// On success, the caller must update the ITransactionalKey.LockType so the unlock has the right type + bool TryPromoteLock(TTransactionalKey key) + where TTransactionalKey : ITransactionalKey; /// /// Promotes a shared lock on the key to an exclusive lock, with retry limits or cancellation. /// - /// + /// /// key whose lock is to be promoted. /// The cancellation token - /// On success, the caller must update the ILockableKey.LockType so the unlock has the right type - bool TryPromoteLock(TLockableKey key, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + /// On success, the caller must update the ITransactionalKey.LockType so the unlock has the right type + bool TryPromoteLock(TTransactionalKey key, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Promotes a shared lock on the key to an exclusive lock, with retry limits or cancellation. /// - /// + /// /// key whose lock is to be promoted. /// TimeSpan limiting the duration of the TryPromoteLock() call. /// The cancellation token, if any - /// On success, the caller must update the ILockableKey.LockType so the unlock has the right type - bool TryPromoteLock(TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey; + /// On success, the caller must update the ITransactionalKey.LockType so the unlock has the right type + bool TryPromoteLock(TTransactionalKey key, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey; /// /// Unlocks the keys identified in the passed array. /// - /// - /// key hashCodes to be unlocked, and whether that unlocking is shared or exclusive; must be sorted by . - void Unlock(TLockableKey[] keys) - where TLockableKey : ILockableKey; + /// + /// key hashCodes to be unlocked, and whether that unlocking is shared or exclusive; must be sorted by . + void Unlock(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey; /// /// Unlocks the keys identified in the passed array. /// - /// - /// key hashCodes to be unlocked, and whether that unlocking is shared or exclusive; must be sorted by . + /// + /// key hashCodes to be unlocked, and whether that unlocking is shared or exclusive; must be sorted by . /// The starting index to Unlock /// The number of keys to Unlock - void Unlock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey; + void Unlock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ITsavoriteContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITsavoriteContext.cs index b4218722e0b..ce5c33fc308 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ITsavoriteContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ITsavoriteContext.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Threading; using System.Threading.Tasks; @@ -9,30 +10,23 @@ namespace Tsavorite.core /// /// Interface for Key-only Tsavorite operations /// - public interface ITsavoriteContext + public interface ITsavoriteContext { /// - /// Obtain a code by which groups of keys will be sorted for manual locking, to avoid deadlocks. + /// Obtain a code by which groups of keys will be sorted for Transactional locking, to avoid deadlocks. /// The key to obtain a code for /// - /// The hashcode of the key; created and returned by - long GetKeyHash(TKey key); - - /// - /// Obtain a code by which groups of keys will be sorted for manual locking, to avoid deadlocks. - /// The key to obtain a code for - /// - /// The hashcode of the key; created and returned by - long GetKeyHash(ref TKey key); + /// The hashcode of the key; created and returned by + long GetKeyHash(ReadOnlySpan key); } /// /// Interface for Tsavorite operations /// - public interface ITsavoriteContext : ITsavoriteContext - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public interface ITsavoriteContext : ITsavoriteContext + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Indicates whether this context has been initialized. @@ -40,9 +34,9 @@ public interface ITsavoriteContext - /// Obtain the underlying + /// Obtain the underlying /// - ClientSession Session { get; } + ClientSession Session { get; } /// /// Synchronously complete outstanding pending synchronous operations. @@ -61,7 +55,7 @@ public interface ITsavoriteContextWait for all pending operations on session to complete /// Spin-wait until ongoing commit/checkpoint, if any, completes /// True if all pending operations have completed, false otherwise - bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false); + bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false); /// /// Complete all pending synchronous Tsavorite operations. @@ -75,7 +69,7 @@ public interface ITsavoriteContext /// Outputs completed by this operation - ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default); + ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default); /// /// Read operation @@ -84,8 +78,8 @@ public interface ITsavoriteContextInput to help extract the retrieved value into /// The location to place the retrieved value /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default); + /// is populated by the implementation + Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default); /// /// Read operation @@ -95,75 +89,35 @@ public interface ITsavoriteContextThe location to place the retrieved value /// Contains options controlling the Read operation /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default); + /// is populated by the implementation + Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default); /// /// Read operation /// /// The key to look up - /// Input to help extract the retrieved value into /// The location to place the retrieved value /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(TKey key, TInput input, out TOutput output, TContext userContext = default); + /// is populated by the implementation + Status Read(ReadOnlySpan key, ref TOutput output, TContext userContext = default); /// /// Read operation /// /// The key to look up - /// Input to help extract the retrieved value into /// The location to place the retrieved value /// Contains options controlling the Read operation /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(TKey key, TInput input, out TOutput output, ref ReadOptions readOptions, TContext userContext = default); - - /// - /// Read operation - /// - /// The key to look up - /// The location to place the retrieved value - /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(ref TKey key, ref TOutput output, TContext userContext = default); - - /// - /// Read operation - /// - /// The key to look up - /// The location to place the retrieved value - /// Contains options controlling the Read operation - /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(ref TKey key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default); - - /// - /// Read operation - /// - /// - /// - /// - /// - Status Read(TKey key, out TOutput output, TContext userContext = default); + /// is populated by the implementation + Status Read(ReadOnlySpan key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default); /// /// Read operation /// /// - /// - /// Contains options controlling the Read operation /// /// - Status Read(TKey key, out TOutput output, ref ReadOptions readOptions, TContext userContext = default); - - /// - /// Read operation - /// - /// - /// - /// - public (Status status, TOutput output) Read(TKey key, TContext userContext = default); + public (Status status, TOutput output) Read(ReadOnlySpan key, TContext userContext = default); /// /// Read operation @@ -172,7 +126,7 @@ public interface ITsavoriteContextContains options controlling the Read operation /// /// - public (Status status, TOutput output) Read(TKey key, ref ReadOptions readOptions, TContext userContext = default); + public (Status status, TOutput output) Read(ReadOnlySpan key, ref ReadOptions readOptions, TContext userContext = default); /// /// Read operation that accepts a ref argument to start the lookup at instead of starting at the hash table entry for , @@ -184,15 +138,15 @@ public interface ITsavoriteContextContains options controlling the Read operation /// On output, receives: /// - ///
  • The address of the found record. This may be different from the passed on the call, due to - /// tracing back over hash collisions until we arrive at the key match
  • - ///
  • A copy of the record's header in ; can be passed - /// in a subsequent call, thereby enumerating all records in a key's hash chain.
  • + /// The address of the found record. This may be different from the passed on the call, due to + /// tracing back over hash collisions until we arrive at the key match + /// A copy of the record's header in ; can be passed + /// in a subsequent call, thereby enumerating all records in a key's hash chain. ///
    /// /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation - Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default); + /// is populated by the implementation + Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// /// Read operation that accepts an address to lookup at, instead of a key. @@ -203,7 +157,7 @@ public interface ITsavoriteContextContains options controlling the Read operation, including the address to read at in StartAddress /// On output, receives metadata about the record /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation; this should store the key if it needs it + /// is populated by the implementation; this should store the key if it needs it Status ReadAtAddress(long address, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// @@ -216,8 +170,8 @@ public interface ITsavoriteContextContains options controlling the Read operation, including the address to read at in StartAddress /// On output, receives metadata about the record /// User application context passed in case the read goes pending due to IO - /// is populated by the implementation; this should store the key if it needs it - Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default); + /// is populated by the implementation; this should store the key if it needs it + Status ReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// /// Upsert operation @@ -226,7 +180,7 @@ public interface ITsavoriteContext /// /// - Status Upsert(ref TKey key, ref TValue desiredValue, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, TContext userContext = default); /// /// Upsert operation @@ -236,7 +190,7 @@ public interface ITsavoriteContext /// /// - Status Upsert(ref TKey key, ref TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default); /// /// Upsert operation @@ -247,7 +201,7 @@ public interface ITsavoriteContext /// /// - Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default); /// /// Upsert operation @@ -259,7 +213,7 @@ public interface ITsavoriteContext /// /// - Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default); /// /// Upsert operation @@ -268,42 +222,41 @@ public interface ITsavoriteContext /// /// + /// /// /// /// - Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// /// Upsert operation /// /// - /// /// - /// - /// - /// /// /// - Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default); + Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, TContext userContext = default); /// /// Upsert operation /// /// /// + /// /// /// - Status Upsert(TKey key, TValue desiredValue, TContext userContext = default); + Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default); /// /// Upsert operation /// /// + /// /// - /// + /// /// /// - Status Upsert(TKey key, TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default); /// /// Upsert operation @@ -312,9 +265,10 @@ public interface ITsavoriteContext /// /// + /// /// /// - Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default); /// /// Upsert operation @@ -324,53 +278,43 @@ public interface ITsavoriteContext /// /// + /// /// /// - Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// - /// RMW operation + /// Upsert operation with a disk log record /// - /// - /// - /// - /// + /// Log record that was read from disk /// - Status RMW(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default); + Status Upsert(ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord; /// - /// RMW operation + /// Upsert operation with a disk log record /// - /// - /// - /// - /// - /// + /// Key, which may be from or may be a modified key (e.g. prepending a prefix) + /// Log record that was read from disk /// - Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord; /// - /// RMW operation + /// Upsert operation with a disk log record and user-supplied key /// - /// - /// - /// - /// - /// + /// Log record that was read from disk /// - Status RMW(ref TKey key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default); + Status Upsert(ref TInput input, ref TSourceLogRecord diskLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord; /// - /// RMW operation + /// Upsert operation with a disk log record and user-supplied key /// - /// - /// - /// - /// - /// - /// + /// Log record that was read from disk /// - Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default); + Status Upsert(ReadOnlySpan key, ref TInput input, ref TSourceLogRecord diskLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord; /// /// RMW operation @@ -380,7 +324,7 @@ public interface ITsavoriteContext /// /// - Status RMW(TKey key, TInput input, out TOutput output, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default); /// /// RMW operation @@ -391,26 +335,30 @@ public interface ITsavoriteContext /// /// - Status RMW(TKey key, TInput input, out TOutput output, ref RMWOptions rmwOptions, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default); /// /// RMW operation /// /// /// + /// + /// /// /// - Status RMW(ref TKey key, ref TInput input, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default); /// /// RMW operation /// /// /// + /// /// + /// /// /// - Status RMW(ref TKey key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default); /// /// RMW operation @@ -419,7 +367,7 @@ public interface ITsavoriteContext /// /// - Status RMW(TKey key, TInput input, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, TContext userContext = default); /// /// RMW operation @@ -429,24 +377,7 @@ public interface ITsavoriteContext /// /// - Status RMW(TKey key, TInput input, ref RMWOptions rmwOptions, TContext userContext = default); - - /// - /// Delete operation - /// - /// - /// - /// - Status Delete(ref TKey key, TContext userContext = default); - - /// - /// Delete operation - /// - /// - /// - /// - /// - Status Delete(ref TKey key, ref DeleteOptions deleteOptions, TContext userContext = default); + Status RMW(ReadOnlySpan key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default); /// /// Delete operation @@ -454,7 +385,7 @@ public interface ITsavoriteContext /// /// - Status Delete(TKey key, TContext userContext = default); + Status Delete(ReadOnlySpan key, TContext userContext = default); /// /// Delete operation @@ -463,13 +394,13 @@ public interface ITsavoriteContext /// /// - Status Delete(TKey key, ref DeleteOptions deleteOptions, TContext userContext = default); + Status Delete(ReadOnlySpan key, ref DeleteOptions deleteOptions, TContext userContext = default); /// /// Reset the modified bit of a record (for in memory records) /// /// - void ResetModified(ref TKey key); + void ResetModified(ReadOnlySpan key); /// /// Refresh session epoch and handle checkpointing phases. Used only diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/IUnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/IUnsafeContext.cs index 8e1ef26f327..20fbd6f5beb 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/IUnsafeContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/IUnsafeContext.cs @@ -5,8 +5,8 @@ namespace Tsavorite.core { /// /// Manual epoch control functions. Useful when doing generic operations across diverse - /// and - /// specializations. + /// and + /// specializations. /// public interface IUnsafeContext { diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs deleted file mode 100644 index e9eeb06eab3..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs +++ /dev/null @@ -1,517 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Diagnostics; -using System.Runtime.CompilerServices; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// Tsavorite Context implementation that allows manual control of record locking and epoch management. For advanced use only. - /// - public readonly struct LockableUnsafeContext - : ITsavoriteContext, ILockableContext, IUnsafeContext - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - readonly ClientSession clientSession; - readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; - - /// - public bool IsNull => clientSession is null; - - internal LockableUnsafeContext(ClientSession clientSession) - { - this.clientSession = clientSession; - sessionFunctions = new(clientSession); - } - - #region Begin/EndUnsafe - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void BeginUnsafe() => clientSession.UnsafeResumeThread(sessionFunctions); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void EndUnsafe() => clientSession.UnsafeSuspendThread(); - - #endregion Begin/EndUnsafe - - #region Begin/EndLockable - - /// - public void BeginLockable() => clientSession.AcquireLockable(sessionFunctions); - - /// - public void LocksAcquired(long txnVersion) => clientSession.LocksAcquired(sessionFunctions, txnVersion); - - /// - public void EndLockable() => clientSession.ReleaseLockable(sessionFunctions); - #endregion Begin/EndLockable - - #region Key Locking - - /// - public int CompareKeyHashes(TLockableKey key1, TLockableKey key2) where TLockableKey : ILockableKey => clientSession.CompareKeyHashes(key1, key2); - - /// - public int CompareKeyHashes(ref TLockableKey key1, ref TLockableKey key2) where TLockableKey : ILockableKey => clientSession.CompareKeyHashes(ref key1, ref key2); - - /// - public void SortKeyHashes(TLockableKey[] keys) where TLockableKey : ILockableKey => clientSession.SortKeyHashes(keys); - - /// - public void SortKeyHashes(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey => clientSession.SortKeyHashes(keys, start, count); - - /// - public void Lock(TLockableKey[] keys) where TLockableKey : ILockableKey => Lock(keys, 0, keys.Length); - - /// - public void Lock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey - { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); - while (true) - { - if (LockableContext.DoManualLock(sessionFunctions, clientSession, keys, start, count)) - { - break; - } - // Suspend and resume epoch protection to give others a fair chance to progress - clientSession.store.epoch.Suspend(); - clientSession.store.epoch.Resume(); - } - } - - /// - public bool TryLock(TLockableKey[] keys) - where TLockableKey : ILockableKey - => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken: default); - - /// - public bool TryLock(TLockableKey[] keys, TimeSpan timeout) - where TLockableKey : ILockableKey - => TryLock(keys, 0, keys.Length, timeout, cancellationToken: default); - - /// - public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout) - where TLockableKey : ILockableKey - => TryLock(keys, start, count, timeout, cancellationToken: default); - - /// - public bool TryLock(TLockableKey[] keys, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken); - - /// - public bool TryLock(TLockableKey[] keys, int start, int count, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - => TryLock(keys, start, count, Timeout.InfiniteTimeSpan, cancellationToken); - - /// - public bool TryLock(TLockableKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - => TryLock(keys, 0, keys.Length, timeout, cancellationToken); - - /// - public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); - - return LockableContext.DoManualTryLock(sessionFunctions, clientSession, keys, start, count, timeout, cancellationToken); - } - - /// - public bool TryPromoteLock(TLockableKey key) - where TLockableKey : ILockableKey - => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken: default); - - /// - public bool TryPromoteLock(TLockableKey key, TimeSpan timeout) - where TLockableKey : ILockableKey - => TryPromoteLock(key, timeout, cancellationToken: default); - - /// - public bool TryPromoteLock(TLockableKey key, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken); - - /// - public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey - { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); - - return LockableContext.DoManualTryPromoteLock(sessionFunctions, clientSession, key, timeout, cancellationToken); - } - - /// - public void Unlock(TLockableKey[] keys) where TLockableKey : ILockableKey => Unlock(keys, 0, keys.Length); - - /// - public void Unlock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey - { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Unlock()"); - - LockableContext.DoManualUnlock(clientSession, keys, start, start + count - 1); - } - - /// - /// The id of the current Tsavorite Session - /// - public int SessionID { get { return clientSession.ctx.sessionID; } } - - #endregion Key Locking - - #region ITsavoriteContext - - /// - public ClientSession Session => clientSession; - - /// - public long GetKeyHash(TKey key) => clientSession.store.GetKeyHash(ref key); - - /// - public long GetKeyHash(ref TKey key) => clientSession.store.GetKeyHash(ref key); - - /// - public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.UnsafeCompletePending(sessionFunctions, false, wait, spinWaitForCommit); - } - - /// - public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.UnsafeCompletePendingWithOutputs(sessionFunctions, out completedOutputs, wait, spinWaitForCommit); - } - - /// - public ValueTask CompletePendingAsync(bool waitForCommit = false, CancellationToken token = default) - => clientSession.CompletePendingAsync(sessionFunctions, waitForCommit, token); - - /// - public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) - => clientSession.CompletePendingWithOutputsAsync(sessionFunctions, waitForCommit, token); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, ref readOptions, out _, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, TContext userContext = default) - { - TInput input = default; - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, TContext userContext = default) - { - TInput input = default; - TOutput output = default; - return (Read(ref key, ref input, ref output, userContext), output); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - TOutput output = default; - return (Read(ref key, ref input, ref output, ref readOptions, userContext), output); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status ReadAtAddress(long address, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextReadAtAddress(address, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, TContext userContext = default) - { - TInput input = default; - TOutput output = default; - return Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) - { - TInput input = default; - TOutput output = default; - return Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, TContext userContext = default) - => Upsert(ref key, ref desiredValue, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref desiredValue, ref upsertOptions, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, ref upsertOptions, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) - => RMW(ref key, ref input, ref output, out _, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out _, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRMW(ref key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, ref RMWOptions rmwOptions, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, TContext userContext = default) - => Delete(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); - - /// - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, long keyHash, TContext userContext = default) - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( - ref key, keyHash, userContext, sessionFunctions); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, TContext userContext = default) - => Delete(ref key, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, ref deleteOptions, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ResetModified(ref TKey key) - => clientSession.UnsafeResetModified(sessionFunctions, ref key); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool IsModified(TKey key) - => clientSession.UnsafeIsModified(sessionFunctions, ref key); - - /// - public void Refresh() - { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); - } - - #endregion ITsavoriteContext - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs index e98466b133c..a0a40c7b6c5 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs @@ -7,9 +7,9 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal Dictionary _activeSessions = new(); @@ -19,8 +19,9 @@ public unsafe partial class TsavoriteKVCallback functions /// for this session; override those specified at TsavoriteKV level, and may be overridden on individual Read operations /// Session instance - public ClientSession NewSession(TFunctions functions, ReadCopyOptions readCopyOptions = default) - where TFunctions : ISessionFunctions + public ClientSession NewSession(TFunctions functions, + ReadCopyOptions readCopyOptions = default) + where TFunctions : ISessionFunctions { if (functions == null) throw new ArgumentNullException(nameof(functions)); @@ -29,16 +30,14 @@ public ClientSession(sessionID); ctx.MergeReadCopyOptions(ReadCopyOptions, readCopyOptions); - - var session = new ClientSession(this, ctx, functions); if (RevivificationManager.IsEnabled) { if (_activeSessions == null) _ = Interlocked.CompareExchange(ref _activeSessions, [], null); - - lock (_activeSessions) - _activeSessions.Add(sessionID, new SessionInfo { session = session, isActive = true }); } + var session = new ClientSession(this, ctx, functions); + lock (_activeSessions) + _activeSessions.Add(sessionID, new SessionInfo { session = session, isActive = true }); return session; } diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs new file mode 100644 index 00000000000..463c9c1d17b --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/NoOpSessionFunctions.cs @@ -0,0 +1,95 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Tsavorite.core +{ + /// + /// This implementation of is used during compaction, iteration, and any other + /// operations that require a session for ContinuePending but do operations directly on the rather than calling + /// methods for record operations (Delete methods simply return true to let + /// Tsavorite proceed with the delete). + /// + /// + /// + /// + internal struct NoOpSessionFunctions : ISessionFunctions + { + public readonly bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; + + public readonly void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { } + + public readonly bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) => true; + + public readonly bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) => true; + + public readonly bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) => true; + + public readonly bool InPlaceWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public readonly bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public readonly bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public readonly bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; + public readonly void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) { } + + public readonly bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; + + public readonly bool NeedInitialUpdate(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; + + public readonly bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public readonly void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } + + public readonly void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } + + public readonly RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord + => default; + public readonly RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TInput input) => default; + public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref TInput input) => default; + public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input) => default; + public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord + => default; + + /// + /// No reads during compaction + /// + public readonly bool Reader(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput dst, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + /// + /// Write compacted live value to store + /// + public readonly bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) => true; + + /// + /// Write compacted live value to store + /// + public readonly bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) => true; + + public readonly bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + => true; + + public readonly void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) { } + public readonly void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) { } + public readonly void PostInitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { } + + public readonly void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs index 49feed18b1a..b1ea793f662 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/SessionFunctionsWrapper.cs @@ -1,63 +1,102 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Runtime.CompilerServices; namespace Tsavorite.core { - internal readonly struct SessionFunctionsWrapper - : ISessionFunctionsWrapper - where TFunctions : ISessionFunctions - where TSessionLocker : struct, ISessionLocker - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal readonly struct SessionFunctionsWrapper + : ISessionFunctionsWrapper + where TFunctions : ISessionFunctions + where TSessionLocker : struct, ISessionLocker + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - private readonly ClientSession _clientSession; + private readonly ClientSession _clientSession; private readonly TSessionLocker _sessionLocker; // Has no data members - public SessionFunctionsWrapper(ClientSession clientSession) + public SessionFunctionsWrapper(ClientSession clientSession) { _clientSession = clientSession; _sessionLocker = new TSessionLocker(); } - public TsavoriteKV Store => _clientSession.store; - public OverflowBucketLockTable LockTable => _clientSession.store.LockTable; + public TsavoriteKV Store => _clientSession.store; + public OverflowBucketLockTable LockTable => _clientSession.store.LockTable; #region Reads [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo) - => _clientSession.functions.SingleReader(ref key, ref input, ref value, ref dst, ref readInfo); + public bool Reader(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput dst, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.Reader(ref srcLogRecord, ref input, ref dst, ref readInfo); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - => _clientSession.functions.ConcurrentReader(ref key, ref input, ref value, ref dst, ref readInfo, ref recordInfo); - - public void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) - => _clientSession.functions.ReadCompletionCallback(ref key, ref input, ref output, ctx, status, recordMetadata); + public void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) + => _clientSession.functions.ReadCompletionCallback(ref diskLogRecord, ref input, ref output, ctx, status, recordMetadata); #endregion Reads #region Upserts [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - => _clientSession.functions.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + public bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + => _clientSession.functions.InitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + => _clientSession.functions.InitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + logRecord.InfoRef.SetDirtyAndModified(); + _clientSession.functions.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + logRecord.InfoRef.SetDirtyAndModified(); + _clientSession.functions.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public readonly void PostInitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord { - recordInfo.SetDirtyAndModified(); - _clientSession.functions.PostSingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason); + dstLogRecord.InfoRef.SetDirtyAndModified(); + _clientSession.functions.PostInitialWriter(ref dstLogRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool ConcurrentWriter(long physicalAddress, ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) { - (upsertInfo.UsedValueLength, upsertInfo.FullValueLength, _) = _clientSession.store.GetRecordLengths(physicalAddress, ref dst, ref recordInfo); - if (!_clientSession.functions.ConcurrentWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo)) + if (!_clientSession.functions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo)) return false; - _clientSession.store.SetExtraValueLength(ref dst, ref recordInfo, upsertInfo.UsedValueLength, upsertInfo.FullValueLength); - recordInfo.SetDirtyAndModified(); + logRecord.InfoRef.SetDirtyAndModified(); + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + if (!_clientSession.functions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo)) + return false; + logRecord.InfoRef.SetDirtyAndModified(); + return true; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + if (!_clientSession.functions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo)) + return false; + logRecord.InfoRef.SetDirtyAndModified(); return true; } #endregion Upserts @@ -65,49 +104,50 @@ public bool ConcurrentWriter(long physicalAddress, ref TKey key, ref TInput inpu #region RMWs #region InitialUpdater [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) - => _clientSession.functions.NeedInitialUpdate(ref key, ref input, ref output, ref rmwInfo); + public bool NeedInitialUpdate(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + => _clientSession.functions.NeedInitialUpdate(key, ref input, ref output, ref rmwInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - => _clientSession.functions.InitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + public bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + => _clientSession.functions.InitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) { - recordInfo.SetDirtyAndModified(); - _clientSession.functions.PostInitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo); + logRecord.InfoRef.SetDirtyAndModified(); + _clientSession.functions.PostInitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); } #endregion InitialUpdater #region CopyUpdater [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo) - => _clientSession.functions.NeedCopyUpdate(ref key, ref input, ref oldValue, ref output, ref rmwInfo); + public bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.NeedCopyUpdate(ref srcLogRecord, ref input, ref output, ref rmwInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - => _clientSession.functions.CopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo, ref recordInfo); + public bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.CopyUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord { - recordInfo.SetDirtyAndModified(); - return _clientSession.functions.PostCopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo); + dstLogRecord.InfoRef.SetDirtyAndModified(); + return _clientSession.functions.PostCopyUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); } #endregion CopyUpdater #region InPlaceUpdater [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool InPlaceUpdater(long physicalAddress, ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, out OperationStatus status, ref RecordInfo recordInfo) + public bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo, out OperationStatus status) { - (rmwInfo.UsedValueLength, rmwInfo.FullValueLength, rmwInfo.FullRecordLength) = _clientSession.store.GetRecordLengths(physicalAddress, ref value, ref recordInfo); - - if (_clientSession.functions.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo)) + // This wraps the ISessionFunctions call to provide expiration logic. + if (_clientSession.functions.InPlaceUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) { rmwInfo.Action = RMWAction.Default; - _clientSession.store.SetExtraValueLength(ref value, ref recordInfo, rmwInfo.UsedValueLength, rmwInfo.FullValueLength); - recordInfo.SetDirtyAndModified(); + logRecord.InfoRef.SetDirtyAndModified(); // MarkPage is done in InternalRMW status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); @@ -117,8 +157,8 @@ public bool InPlaceUpdater(long physicalAddress, ref TKey key, ref TInput input, if (rmwInfo.Action == RMWAction.ExpireAndResume) { // This inserts the tombstone if appropriate - return _clientSession.store.ReinitializeExpiredRecord>( - ref key, ref input, ref value, ref output, ref recordInfo, ref rmwInfo, rmwInfo.Address, this, isIpu: true, out status); + return _clientSession.store.ReinitializeExpiredRecord>( + ref logRecord, ref input, ref output, ref rmwInfo, rmwInfo.Address, this, isIpu: true, out status); } if (rmwInfo.Action == RMWAction.CancelOperation) @@ -127,7 +167,7 @@ public bool InPlaceUpdater(long physicalAddress, ref TKey key, ref TInput input, } else if (rmwInfo.Action == RMWAction.ExpireAndStop) { - recordInfo.SetTombstone(); + logRecord.InfoRef.SetTombstone(); status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord | StatusCode.Expired); } else @@ -136,31 +176,30 @@ public bool InPlaceUpdater(long physicalAddress, ref TKey key, ref TInput input, } #endregion InPlaceUpdater - public void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) - => _clientSession.functions.RMWCompletionCallback(ref key, ref input, ref output, ctx, status, recordMetadata); + public void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) + => _clientSession.functions.RMWCompletionCallback(ref diskLogRecord, ref input, ref output, ctx, status, recordMetadata); #endregion RMWs #region Deletes [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) - => _clientSession.functions.SingleDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + public bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + => _clientSession.functions.InitialDeleter(ref logRecord, ref deleteInfo); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - recordInfo.SetDirtyAndModified(); - _clientSession.functions.PostSingleDeleter(ref key, ref deleteInfo); + logRecord.InfoRef.SetDirtyAndModified(); + _clientSession.functions.PostInitialDeleter(ref logRecord, ref deleteInfo); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool ConcurrentDeleter(long physicalAddress, ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo, out int allocatedSize) + public bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { - (deleteInfo.UsedValueLength, deleteInfo.FullValueLength, allocatedSize) = _clientSession.store.GetRecordLengths(physicalAddress, ref value, ref recordInfo); - if (!_clientSession.functions.ConcurrentDeleter(ref key, ref value, ref deleteInfo, ref recordInfo)) + if (!_clientSession.functions.InPlaceDeleter(ref logRecord, ref deleteInfo)) return false; - _clientSession.store.SetTombstoneAndExtraValueLength(ref value, ref recordInfo, deleteInfo.UsedValueLength, deleteInfo.FullValueLength); - recordInfo.SetDirtyAndModified(); + logRecord.InfoRef.SetTombstone(); + logRecord.InfoRef.SetDirtyAndModified(); return true; } #endregion Deletes @@ -170,43 +209,45 @@ public bool ConcurrentDeleter(long physicalAddress, ref TKey key, ref TValue val public void ConvertOutputToHeap(ref TInput input, ref TOutput output) => _clientSession.functions.ConvertOutputToHeap(ref input, ref output); #endregion Utilities - #region Transient locking - public bool IsManualLocking => _sessionLocker.IsManualLocking; + #region Ephemeral locking + public bool IsTransactionalLocking => _sessionLocker.IsTransactionalLocking; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientExclusive(ref TKey key, ref OperationStackContext stackCtx) => - _sessionLocker.TryLockTransientExclusive(Store, ref stackCtx); + public bool TryLockEphemeralExclusive(ref OperationStackContext stackCtx) => + _sessionLocker.TryLockEphemeralExclusive(Store, ref stackCtx); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientShared(ref TKey key, ref OperationStackContext stackCtx) - => _sessionLocker.TryLockTransientShared(Store, ref stackCtx); + public bool TryLockEphemeralShared(ref OperationStackContext stackCtx) + => _sessionLocker.TryLockEphemeralShared(Store, ref stackCtx); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientExclusive(ref TKey key, ref OperationStackContext stackCtx) - => _sessionLocker.UnlockTransientExclusive(Store, ref stackCtx); + public void UnlockEphemeralExclusive(ref OperationStackContext stackCtx) + => _sessionLocker.UnlockEphemeralExclusive(Store, ref stackCtx); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientShared(ref TKey key, ref OperationStackContext stackCtx) - => _sessionLocker.UnlockTransientShared(Store, ref stackCtx); - #endregion Transient locking + public void UnlockEphemeralShared(ref OperationStackContext stackCtx) + => _sessionLocker.UnlockEphemeralShared(Store, ref stackCtx); + #endregion Ephemeral locking #region Internal utilities [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetRMWInitialValueLength(ref TInput input) => _clientSession.functions.GetRMWInitialValueLength(ref input); + public RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TInput input) => _clientSession.functions.GetRMWInitialFieldInfo(key, ref input); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetRMWModifiedValueLength(ref TValue t, ref TInput input) => _clientSession.functions.GetRMWModifiedValueLength(ref t, ref input); + public RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.GetRMWModifiedFieldInfo(ref srcLogRecord, ref input); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int GetUpsertValueLength(ref TValue t, ref TInput input) => _clientSession.functions.GetUpsertValueLength(ref t, ref input); + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref TInput input) => _clientSession.functions.GetUpsertFieldInfo(key, value, ref input); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public IHeapContainer GetHeapContainer(ref TInput input) - { - if (typeof(TInput) == typeof(SpanByte)) - return new SpanByteHeapContainer(ref Unsafe.As(ref input), _clientSession.store.hlogBase.bufferPool) as IHeapContainer; - return new StandardHeapContainer(ref input); - } + public RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input) => _clientSession.functions.GetUpsertFieldInfo(key, value, ref input); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord + => _clientSession.functions.GetUpsertFieldInfo(key, ref inputLogRecord, ref input); [MethodImpl(MethodImplOptions.AggressiveInlining)] public void UnsafeResumeThread() => _clientSession.UnsafeResumeThread(this); @@ -215,10 +256,10 @@ public IHeapContainer GetHeapContainer(ref TInput input) public void UnsafeSuspendThread() => _clientSession.UnsafeSuspendThread(); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) => _clientSession.CompletePendingWithOutputs(this, out completedOutputs, wait, spinWaitForCommit); - public TsavoriteKV.TsavoriteExecutionContext Ctx => _clientSession.ctx; + public TsavoriteKV.TsavoriteExecutionContext Ctx => _clientSession.ctx; #endregion Internal utilities } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalContext.cs similarity index 52% rename from libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs rename to libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalContext.cs index cc866ccfcde..45aad909d48 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalContext.cs @@ -10,59 +10,59 @@ namespace Tsavorite.core { /// - /// Tsavorite Context implementation that allows manual control of record locking and epoch management. For advanced use only. + /// Tsavorite Context implementation that allows Transactional control of locking and automatic epoch management. For advanced use only. /// - public readonly struct LockableContext : ITsavoriteContext, ILockableContext - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public readonly struct TransactionalContext : ITsavoriteContext, ITransactionalContext + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - readonly ClientSession clientSession; - readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; + readonly ClientSession clientSession; + readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; /// public bool IsNull => clientSession is null; const int KeyLockMaxRetryAttempts = 1000; - internal LockableContext(ClientSession clientSession) + internal TransactionalContext(ClientSession clientSession) { this.clientSession = clientSession; sessionFunctions = new(clientSession); } - #region Begin/EndLockable + #region Begin/EndTransaction /// - public void BeginLockable() => clientSession.AcquireLockable(sessionFunctions); + public void BeginTransaction() => clientSession.AcquireTransactional(sessionFunctions); /// public void LocksAcquired(long txnVersion) => clientSession.LocksAcquired(sessionFunctions, txnVersion); /// - public void EndLockable() => clientSession.ReleaseLockable(sessionFunctions); + public void EndTransaction() => clientSession.ReleaseTransactional(sessionFunctions); - #endregion Begin/EndLockable + #endregion Begin/EndTransaction #region Key Locking /// - public int CompareKeyHashes(TLockableKey key1, TLockableKey key2) where TLockableKey : ILockableKey => clientSession.CompareKeyHashes(ref key1, ref key2); + public int CompareKeyHashes(TTransactionalKey key1, TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => clientSession.CompareKeyHashes(ref key1, ref key2); /// - public int CompareKeyHashes(ref TLockableKey key1, ref TLockableKey key2) where TLockableKey : ILockableKey => clientSession.CompareKeyHashes(ref key1, ref key2); + public int CompareKeyHashes(ref TTransactionalKey key1, ref TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => clientSession.CompareKeyHashes(ref key1, ref key2); /// - public void SortKeyHashes(TLockableKey[] keys) where TLockableKey : ILockableKey => clientSession.SortKeyHashes(keys); + public void SortKeyHashes(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => clientSession.SortKeyHashes(keys); /// - public void SortKeyHashes(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey => clientSession.SortKeyHashes(keys, start, count); + public void SortKeyHashes(TTransactionalKey[] keys, int start, int count) where TTransactionalKey : ITransactionalKey => clientSession.SortKeyHashes(keys, start, count); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool DoManualLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, - TLockableKey[] keys, int start, int count) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - where TLockableKey : ILockableKey + internal static bool DoTransactionalLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, + TTransactionalKey[] keys, int start, int count) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TTransactionalKey : ITransactionalKey { // The key codes are sorted, but there may be duplicates; the sorting is such that exclusive locks come first for each key code, // which of course allows the session to do shared operations as well, so we take the first occurrence of each key code. @@ -80,7 +80,7 @@ internal static bool DoManualLock(TSessi if (currBucketIndex != prevBucketIndex) { prevBucketIndex = currBucketIndex; - OperationStatus status = DoManualLock(clientSession, key); + OperationStatus status = DoTransactionalLock(clientSession, key); if (status == OperationStatus.SUCCESS) continue; // Success; continue to the next key. @@ -101,14 +101,14 @@ internal static bool DoManualLock(TSessi } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool DoManualTryLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, - TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - where TLockableKey : ILockableKey + internal static bool DoTransactionalTryLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, + TTransactionalKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TTransactionalKey : ITransactionalKey { // The key codes are sorted, but there may be duplicates; the sorting is such that exclusive locks come first for each key code, // which of course allows the session to do shared operations as well, so we take the first occurrence of each key code. - // This is the same as DoManualLock but with timeout. + // This is the same as DoTransactionalLock but with timeout. var end = start + count - 1; // We can't start each retry with a full timeout because we might always fail if someone is not unlocking (e.g. another thread hangs @@ -132,7 +132,7 @@ internal static bool DoManualTryLock(TSe status = OperationStatus.CANCELED; else { - status = DoManualLock(clientSession, key); + status = DoTransactionalLock(clientSession, key); if (status == OperationStatus.SUCCESS) continue; // Success; continue to the next key. } @@ -156,10 +156,10 @@ internal static bool DoManualTryLock(TSe } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static bool DoManualTryPromoteLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, - TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - where TLockableKey : ILockableKey + internal static bool DoManualTryPromoteLock(TSessionFunctionsWrapper sessionFunctions, ClientSession clientSession, + TTransactionalKey key, TimeSpan timeout, CancellationToken cancellationToken) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TTransactionalKey : ITransactionalKey { var startTime = DateTime.UtcNow; while (true) @@ -169,7 +169,7 @@ internal static bool DoManualTryPromoteLock(ClientSession clientSession, TLockableKey key) - where TLockableKey : ILockableKey + internal static OperationStatus DoTransactionalLock(ClientSession clientSession, TTransactionalKey key) + where TTransactionalKey : ITransactionalKey { if (key.LockType == LockType.Shared) { @@ -206,9 +206,9 @@ internal static OperationStatus DoManualLock(ClientSession(ClientSession clientSession, - TLockableKey[] keys, int start, int keyIdx) - where TLockableKey : ILockableKey + internal static void DoManualUnlock(ClientSession clientSession, + TTransactionalKey[] keys, int start, int keyIdx) + where TTransactionalKey : ITransactionalKey { // The key codes are sorted, but there may be duplicates; the sorting is such that exclusive locks come first for each key code. // Unlock has to be done in the reverse order of locking, so we take the *last* occurrence of each key there, and keyIdx moves backward. @@ -233,21 +233,21 @@ internal static void DoManualUnlock(ClientSession - public void Lock(TLockableKey[] keys) where TLockableKey : ILockableKey => Lock(keys, 0, keys.Length); + public void Lock(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => Lock(keys, 0, keys.Length); /// - public void Lock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey + public void Lock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for TransactionalUnsafeContext.Lock()"); bool lockAquired = false; while (!lockAquired) { clientSession.UnsafeResumeThread(sessionFunctions); try { - lockAquired = DoManualLock(sessionFunctions, clientSession, keys, start, count); + lockAquired = DoTransactionalLock(sessionFunctions, clientSession, keys, start, count); } finally { @@ -257,46 +257,46 @@ public void Lock(TLockableKey[] keys, int start, int count) } /// - public bool TryLock(TLockableKey[] keys) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken: default); /// - public bool TryLock(TLockableKey[] keys, TimeSpan timeout) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey => TryLock(keys, 0, keys.Length, timeout, cancellationToken: default); /// - public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey => TryLock(keys, start, count, timeout, cancellationToken: default); /// - public bool TryLock(TLockableKey[] keys, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken); /// - public bool TryLock(TLockableKey[] keys, int start, int count, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, int start, int count, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey => TryLock(keys, start, count, Timeout.InfiniteTimeSpan, cancellationToken); /// - public bool TryLock(TLockableKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey => TryLock(keys, 0, keys.Length, timeout, cancellationToken); /// - public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for TransactionalUnsafeContext.TryLock()"); clientSession.UnsafeResumeThread(sessionFunctions); try { - return DoManualTryLock(sessionFunctions, clientSession, keys, start, count, timeout, cancellationToken); + return DoTransactionalTryLock(sessionFunctions, clientSession, keys, start, count, timeout, cancellationToken); } finally { @@ -305,26 +305,26 @@ public bool TryLock(TLockableKey[] keys, int start, int count, Tim } /// - public bool TryPromoteLock(TLockableKey key) - where TLockableKey : ILockableKey + public bool TryPromoteLock(TTransactionalKey key) + where TTransactionalKey : ITransactionalKey => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken: default); /// - public bool TryPromoteLock(TLockableKey key, TimeSpan timeout) - where TLockableKey : ILockableKey + public bool TryPromoteLock(TTransactionalKey key, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey => TryPromoteLock(key, timeout, cancellationToken: default); /// - public bool TryPromoteLock(TLockableKey key, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryPromoteLock(TTransactionalKey key, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken); /// - public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) - where TLockableKey : ILockableKey + public bool TryPromoteLock(TTransactionalKey key, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for TransactionalUnsafeContext.TryPromoteLock()"); clientSession.UnsafeResumeThread(sessionFunctions); try @@ -338,14 +338,14 @@ public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, Can } /// - public void Unlock(TLockableKey[] keys) where TLockableKey : ILockableKey => Unlock(keys, 0, keys.Length); + public void Unlock(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => Unlock(keys, 0, keys.Length); /// - public void Unlock(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey + public void Unlock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey { - clientSession.CheckIsAcquiredLockable(sessionFunctions); - Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Unlock()"); + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for TransactionalUnsafeContext.Unlock()"); clientSession.UnsafeResumeThread(sessionFunctions); try @@ -368,13 +368,10 @@ public void Unlock(TLockableKey[] keys, int start, int count) #region ITsavoriteContext /// - public ClientSession Session => clientSession; + public ClientSession Session => clientSession; /// - public long GetKeyHash(TKey key) => clientSession.store.GetKeyHash(ref key); - - /// - public long GetKeyHash(ref TKey key) => clientSession.store.GetKeyHash(ref key); + public long GetKeyHash(ReadOnlySpan key) => clientSession.store.GetKeyHash(key); /// public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) @@ -392,7 +389,7 @@ public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) } /// - public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); @@ -411,18 +408,18 @@ public ValueTask CompletePendingAsync(bool waitForCommit = false, CancellationTo => clientSession.CompletePendingAsync(sessionFunctions, waitForCommit, token); /// - public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) + public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) => clientSession.CompletePendingWithOutputsAsync(sessionFunctions, waitForCommit, token); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextRead(ref key, ref input, ref output, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, userContext, sessionFunctions); } finally { @@ -432,86 +429,52 @@ public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - => Read(ref key, ref input, ref output, ref readOptions, out _, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + => Read(key, ref input, ref output, ref readOptions, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, TContext userContext = default) { TInput input = default; - output = default; - return Read(ref key, ref input, ref output, userContext); + return Read(key, ref input, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); + return Read(key, ref input, ref output, ref readOptions, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, userContext), output); + return (Read(key, ref input, ref output, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, ref ReadOptions readOptions, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, ref readOptions, userContext), output); + return (Read(key, ref input, ref output, ref readOptions, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextRead(ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } finally { @@ -537,13 +500,13 @@ public Status ReadAtAddress(long address, ref TInput input, ref TOutput output, /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status ReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextReadAtAddress(address, key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } finally { @@ -553,41 +516,41 @@ public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TO /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, userContext, sessionFunctions); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out _, userContext, sessionFunctions); } finally { @@ -597,23 +560,14 @@ private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue d /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); } finally { @@ -623,53 +577,100 @@ private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue d /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, TContext userContext = default) - => Upsert(ref key, ref desiredValue, userContext); + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref desiredValue, ref upsertOptions, userContext); + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, ref upsertOptions, userContext); + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + { + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); + clientSession.UnsafeResumeThread(sessionFunctions); + try + { + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out _, userContext, sessionFunctions); + } + finally + { + clientSession.UnsafeSuspendThread(); + } + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) - => RMW(ref key, ref input, ref output, out _, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + clientSession.UnsafeResumeThread(sessionFunctions); + try + { + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + } + finally + { + clientSession.UnsafeSuspendThread(); + } + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out _, userContext); + public Status Upsert(ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + => Upsert(diskLogRecord.Key, ref diskLogRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); + public Status Upsert(ReadOnlySpan key, ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + { + TInput input = default; + TOutput output = default; + UpsertOptions upsertOptions = default; + return Upsert(key, ref input, ref diskLogRecord, ref output, ref upsertOptions); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); + public Status Upsert(ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord + => Upsert(inputLogRecord.Key, ref input, ref inputLogRecord, ref output, ref upsertOptions, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextRMW(ref key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextUpsert(key, keyHash, ref input, inputLogRecord: ref inputLogRecord, ref output, out _, userContext, sessionFunctions); } finally { @@ -679,72 +680,76 @@ private Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput out /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) + => RMW(key, ref input, ref output, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + private Status RMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) { - TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); + clientSession.UnsafeResumeThread(sessionFunctions); + try + { + return clientSession.store.ContextRMW(key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); + } + finally + { + clientSession.UnsafeSuspendThread(); + } } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, TContext userContext = default) + public Status RMW(ReadOnlySpan key, ref TInput input, TContext userContext = default) { TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); + return RMW(key, ref input, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + public Status RMW(ReadOnlySpan key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) { TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + return RMW(key, ref input, ref output, ref rmwOptions, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, TContext userContext = default) - => Delete(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status Delete(ReadOnlySpan key, TContext userContext = default) + => Delete(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status Delete(ReadOnlySpan key, ref DeleteOptions deleteOptions, TContext userContext = default) + => Delete(key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Delete(ref TKey key, long keyHash, TContext userContext = default) + private Status Delete(ReadOnlySpan key, long keyHash, TContext userContext = default) { Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected()); clientSession.UnsafeResumeThread(sessionFunctions); try { - return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( - ref key, keyHash, userContext, sessionFunctions); + return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( + key, keyHash, userContext, sessionFunctions); } finally { @@ -754,23 +759,13 @@ private Status Delete(ref TKey key, long keyHash, TContext userContext = default /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, TContext userContext = default) - => Delete(ref key, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, ref deleteOptions, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ResetModified(ref TKey key) - => clientSession.ResetModified(sessionFunctions, ref key); + public void ResetModified(ReadOnlySpan key) + => clientSession.ResetModified(sessionFunctions, key); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool IsModified(TKey key) - => clientSession.IsModified(sessionFunctions, ref key); + internal bool IsModified(ReadOnlySpan key) + => clientSession.IsModified(sessionFunctions, key); /// public void Refresh() @@ -779,7 +774,7 @@ public void Refresh() clientSession.UnsafeResumeThread(sessionFunctions); try { - clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); + clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); } finally { diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalUnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalUnsafeContext.cs new file mode 100644 index 00000000000..7e3f8c6ad61 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/TransactionalUnsafeContext.cs @@ -0,0 +1,485 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Threading; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + /// + /// Tsavorite Context implementation that allows Transactional control of locking and manual epoch management. For advanced use only. + /// + public readonly struct TransactionalUnsafeContext + : ITsavoriteContext, ITransactionalContext, IUnsafeContext + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + readonly ClientSession clientSession; + readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; + + /// + public bool IsNull => clientSession is null; + + internal TransactionalUnsafeContext(ClientSession clientSession) + { + this.clientSession = clientSession; + sessionFunctions = new(clientSession); + } + + #region Begin/EndUnsafe + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void BeginUnsafe() => clientSession.UnsafeResumeThread(sessionFunctions); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void EndUnsafe() => clientSession.UnsafeSuspendThread(); + + #endregion Begin/EndUnsafe + + #region Begin/EndTransaction + + /// + public void BeginTransaction() => clientSession.AcquireTransactional(sessionFunctions); + + /// + public void LocksAcquired(long txnVersion) => clientSession.LocksAcquired(sessionFunctions, txnVersion); + + /// + public void EndTransaction() => clientSession.ReleaseTransactional(sessionFunctions); + #endregion Begin/EndTransaction + + #region Key Locking + + /// + public int CompareKeyHashes(TTransactionalKey key1, TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => clientSession.CompareKeyHashes(key1, key2); + + /// + public int CompareKeyHashes(ref TTransactionalKey key1, ref TTransactionalKey key2) where TTransactionalKey : ITransactionalKey => clientSession.CompareKeyHashes(ref key1, ref key2); + + /// + public void SortKeyHashes(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => clientSession.SortKeyHashes(keys); + + /// + public void SortKeyHashes(TTransactionalKey[] keys, int start, int count) where TTransactionalKey : ITransactionalKey => clientSession.SortKeyHashes(keys, start, count); + + /// + public void Lock(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => Lock(keys, 0, keys.Length); + + /// + public void Lock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey + { + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for TransactionalUnsafeContext.Lock()"); + while (true) + { + if (TransactionalContext.DoTransactionalLock(sessionFunctions, clientSession, keys, start, count)) + { + break; + } + // Suspend and resume epoch protection to give others a fair chance to progress + clientSession.store.epoch.Suspend(); + clientSession.store.epoch.Resume(); + } + } + + /// + public bool TryLock(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken: default); + + /// + public bool TryLock(TTransactionalKey[] keys, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, 0, keys.Length, timeout, cancellationToken: default); + + /// + public bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, start, count, timeout, cancellationToken: default); + + /// + public bool TryLock(TTransactionalKey[] keys, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, 0, keys.Length, Timeout.InfiniteTimeSpan, cancellationToken); + + /// + public bool TryLock(TTransactionalKey[] keys, int start, int count, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, start, count, Timeout.InfiniteTimeSpan, cancellationToken); + + /// + public bool TryLock(TTransactionalKey[] keys, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + => TryLock(keys, 0, keys.Length, timeout, cancellationToken); + + /// + public bool TryLock(TTransactionalKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + { + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for TransactionalUnsafeContext.TryLock()"); + + return TransactionalContext.DoTransactionalTryLock(sessionFunctions, clientSession, keys, start, count, timeout, cancellationToken); + } + + /// + public bool TryPromoteLock(TTransactionalKey key) + where TTransactionalKey : ITransactionalKey + => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken: default); + + /// + public bool TryPromoteLock(TTransactionalKey key, TimeSpan timeout) + where TTransactionalKey : ITransactionalKey + => TryPromoteLock(key, timeout, cancellationToken: default); + + /// + public bool TryPromoteLock(TTransactionalKey key, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + => TryPromoteLock(key, Timeout.InfiniteTimeSpan, cancellationToken); + + /// + public bool TryPromoteLock(TTransactionalKey key, TimeSpan timeout, CancellationToken cancellationToken) + where TTransactionalKey : ITransactionalKey + { + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for TransactionalUnsafeContext.TryPromoteLock()"); + + return TransactionalContext.DoManualTryPromoteLock(sessionFunctions, clientSession, key, timeout, cancellationToken); + } + + /// + public void Unlock(TTransactionalKey[] keys) where TTransactionalKey : ITransactionalKey => Unlock(keys, 0, keys.Length); + + /// + public void Unlock(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey + { + clientSession.CheckIsAcquiredTransactional(sessionFunctions); + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for TransactionalUnsafeContext.Unlock()"); + + TransactionalContext.DoManualUnlock(clientSession, keys, start, start + count - 1); + } + + /// + /// The id of the current Tsavorite Session + /// + public int SessionID { get { return clientSession.ctx.sessionID; } } + + #endregion Key Locking + + #region ITsavoriteContext + + /// + public ClientSession Session => clientSession; + + /// + public long GetKeyHash(ReadOnlySpan key) => clientSession.store.GetKeyHash(key); + + /// + public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.UnsafeCompletePending(sessionFunctions, false, wait, spinWaitForCommit); + } + + /// + public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.UnsafeCompletePendingWithOutputs(sessionFunctions, out completedOutputs, wait, spinWaitForCommit); + } + + /// + public ValueTask CompletePendingAsync(bool waitForCommit = false, CancellationToken token = default) + => clientSession.CompletePendingAsync(sessionFunctions, waitForCommit, token); + + /// + public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) + => clientSession.CompletePendingWithOutputsAsync(sessionFunctions, waitForCommit, token); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextRead(key, ref input, ref output, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextRead(key, ref input, ref output, ref readOptions, out _, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Read(ReadOnlySpan key, ref TOutput output, TContext userContext = default) + { + TInput input = default; + return Read(key, ref input, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Read(ReadOnlySpan key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + { + TInput input = default; + return Read(key, ref input, ref output, ref readOptions, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public (Status status, TOutput output) Read(ReadOnlySpan key, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return (Read(key, ref input, ref output, userContext), output); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public (Status status, TOutput output) Read(ReadOnlySpan key, ref ReadOptions readOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return (Read(key, ref input, ref output, ref readOptions, userContext), output); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextRead(key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status ReadAtAddress(long address, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextReadAtAddress(address, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status ReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextReadAtAddress(address, key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out _, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out _, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + => Upsert(diskLogRecord.Key, ref diskLogRecord); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + { + TInput input = default; + TOutput output = default; + UpsertOptions upsertOptions = default; + return Upsert(key, ref input, ref diskLogRecord, ref output, ref upsertOptions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord + => Upsert(inputLogRecord.Key, ref input, ref inputLogRecord, ref output, ref upsertOptions, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Upsert(ReadOnlySpan key, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, inputLogRecord: ref inputLogRecord, ref output, out _, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) + => RMW(key, ref input, ref output, out _, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out _, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextRMW(key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, TContext userContext = default) + { + TOutput output = default; + return RMW(key, ref input, ref output, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + { + TOutput output = default; + return RMW(key, ref input, ref output, ref rmwOptions, userContext); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Delete(ReadOnlySpan key, TContext userContext = default) + => Delete(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); + + /// + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Delete(ReadOnlySpan key, ref DeleteOptions deleteOptions, TContext userContext = default) + => Delete(key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status Delete(ReadOnlySpan key, long keyHash, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( + key, keyHash, userContext, sessionFunctions); + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void ResetModified(ReadOnlySpan key) + => clientSession.UnsafeResetModified(sessionFunctions, key); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal bool IsModified(ReadOnlySpan key) + => clientSession.UnsafeIsModified(sessionFunctions, key); + + /// + public void Refresh() + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); + } + + #endregion ITsavoriteContext + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/UnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/UnsafeContext.cs index a43fefce038..44bc7cbaec9 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/UnsafeContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/UnsafeContext.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; using System.Threading; @@ -9,21 +10,21 @@ namespace Tsavorite.core { /// - /// Tsavorite Operations implementation that allows manual control of record epoch management. For advanced use only. + /// Tsavorite Operations implementation that allows manual control of epoch management. For advanced use only. /// - public readonly struct UnsafeContext - : ITsavoriteContext, IUnsafeContext - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public readonly struct UnsafeContext + : ITsavoriteContext, IUnsafeContext + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - readonly ClientSession clientSession; - internal readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; + readonly ClientSession clientSession; + internal readonly SessionFunctionsWrapper, TStoreFunctions, TAllocator> sessionFunctions; /// public bool IsNull => clientSession is null; - internal UnsafeContext(ClientSession clientSession) + internal UnsafeContext(ClientSession clientSession) { this.clientSession = clientSession; sessionFunctions = new(clientSession); @@ -44,13 +45,10 @@ internal UnsafeContext(ClientSession - public ClientSession Session => clientSession; + public ClientSession Session => clientSession; /// - public long GetKeyHash(TKey key) => clientSession.store.GetKeyHash(ref key); - - /// - public long GetKeyHash(ref TKey key) => clientSession.store.GetKeyHash(ref key); + public long GetKeyHash(ReadOnlySpan key) => clientSession.store.GetKeyHash(key); /// public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) @@ -60,7 +58,7 @@ public bool CompletePending(bool wait = false, bool spinWaitForCommit = false) } /// - public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) + public bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); return clientSession.UnsafeCompletePendingWithOutputs(sessionFunctions, out completedOutputs, wait, spinWaitForCommit); @@ -71,99 +69,65 @@ public ValueTask CompletePendingAsync(bool waitForCommit = false, CancellationTo => clientSession.CompletePendingAsync(sessionFunctions, waitForCommit, token); /// - public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) + public ValueTask> CompletePendingWithOutputsAsync(bool waitForCommit = false, CancellationToken token = default) => clientSession.CompletePendingWithOutputsAsync(sessionFunctions, waitForCommit, token); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, ref readOptions, out _, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, ref readOptions, out _, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, TInput input, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, TContext userContext = default) { TInput input = default; - return Read(ref key, ref input, ref output, userContext); + return Read(key, ref input, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TOutput output, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); + return Read(key, ref input, ref output, ref readOptions, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, TContext userContext = default) - { - TInput input = default; - output = default; - return Read(ref key, ref input, ref output, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(TKey key, out TOutput output, ref ReadOptions readOptions, TContext userContext = default) - { - TInput input = default; - output = default; - return Read(ref key, ref input, ref output, ref readOptions, userContext); - } - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, userContext), output); + return (Read(key, ref input, ref output, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public (Status status, TOutput output) Read(TKey key, ref ReadOptions readOptions, TContext userContext = default) + public (Status status, TOutput output) Read(ReadOnlySpan key, ref ReadOptions readOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return (Read(ref key, ref input, ref output, ref readOptions, userContext), output); + return (Read(key, ref input, ref output, ref readOptions, userContext), output); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Read(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Read(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRead(ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextRead(key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } /// @@ -176,204 +140,212 @@ public Status ReadAtAddress(long address, ref TInput input, ref TOutput output, /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status ReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) + public Status ReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); + return clientSession.store.ContextReadAtAddress(address, key, ref input, ref output, ref readOptions, out recordMetadata, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out _, userContext); + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ReadOnlySpan desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) { TInput input = default; TOutput output = default; - return Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, TContext userContext = default) + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, userContext, sessionFunctions); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out _, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(ref TKey key, ref TInput input, ref TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => Upsert(ref key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref desiredValue, ref output, out recordMetadata, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, ReadOnlySpan desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcStringValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status Upsert(ref TKey key, long keyHash, ref TInput input, ref TValue desiredValue, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, TContext userContext = default) { - Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextUpsert(ref key, keyHash, ref input, ref desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + TInput input = default; + TOutput output = default; + return Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, TContext userContext = default) - => Upsert(ref key, ref desiredValue, userContext); + public Status Upsert(ReadOnlySpan key, IHeapObject desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) + { + TInput input = default; + TOutput output = default; + return Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TValue desiredValue, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref desiredValue, ref upsertOptions, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + => Upsert(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, ref upsertOptions, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + => Upsert(key, upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, desiredValue, ref output, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Upsert(TKey key, TInput input, TValue desiredValue, ref TOutput output, TContext userContext = default) - => Upsert(ref key, ref input, ref desiredValue, ref output, userContext); + private Status Upsert(ReadOnlySpan key, long keyHash, ref TInput input, IHeapObject desiredValue, ref TOutput output, TContext userContext = default) + { + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out _, userContext, sessionFunctions); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, TContext userContext = default) - => RMW(ref key, ref input, ref output, out _, userContext); + public Status Upsert(ReadOnlySpan key, ref TInput input, IHeapObject desiredValue, ref TOutput output, ref UpsertOptions upsertOptions, out RecordMetadata recordMetadata, TContext userContext = default) + { + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, srcObjectValue: desiredValue, ref output, out recordMetadata, userContext, sessionFunctions); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out _, userContext); + public Status Upsert(ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + => Upsert(diskLogRecord.Key, ref diskLogRecord); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); + public Status Upsert(ReadOnlySpan key, ref TSourceLogRecord diskLogRecord) + where TSourceLogRecord : ISourceLogRecord + { + TInput input = default; + TOutput output = default; + UpsertOptions upsertOptions = default; + return Upsert(key, ref input, ref diskLogRecord, ref output, ref upsertOptions); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) - => RMW(ref key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), ref input, ref output, out recordMetadata, userContext); + public Status Upsert(ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord + => Upsert(inputLogRecord.Key, ref input, ref inputLogRecord, ref output, ref upsertOptions, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + public Status Upsert(ReadOnlySpan key, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertOptions upsertOptions, TContext userContext = default) + where TSourceLogRecord : ISourceLogRecord { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextRMW(ref key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); + var keyHash = upsertOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key); + return clientSession.store.ContextUpsert(key, keyHash, ref input, inputLogRecord: ref inputLogRecord, ref output, out _, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext = default) + => RMW(key, ref input, ref output, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, out TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) - { - output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out _, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, TContext userContext = default) - { - TOutput output = default; - return RMW(ref key, ref input, ref output, userContext); - } + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Status RMW(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWOptions rmwOptions, out RecordMetadata recordMetadata, TContext userContext = default) + => RMW(key, rmwOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), ref input, ref output, out recordMetadata, userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(ref TKey key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + public Status RMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext userContext = default) { - TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); + return clientSession.store.ContextRMW(key, keyHash, ref input, ref output, out recordMetadata, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, TContext userContext = default) - => RMW(ref key, ref input, userContext); + public Status RMW(ReadOnlySpan key, ref TInput input, TContext userContext = default) + { + TOutput output = default; + return RMW(key, ref input, ref output, userContext); + } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status RMW(TKey key, TInput input, ref RMWOptions rmwOptions, TContext userContext = default) + public Status RMW(ReadOnlySpan key, ref TInput input, ref RMWOptions rmwOptions, TContext userContext = default) { TOutput output = default; - return RMW(ref key, ref input, ref output, ref rmwOptions, userContext); + return RMW(key, ref input, ref output, ref rmwOptions, userContext); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, TContext userContext = default) - => Delete(ref key, clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status Delete(ReadOnlySpan key, TContext userContext = default) + => Delete(key, clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(ref key), userContext); + public Status Delete(ReadOnlySpan key, ref DeleteOptions deleteOptions, TContext userContext = default) + => Delete(key, deleteOptions.KeyHash ?? clientSession.store.storeFunctions.GetKeyHashCode64(key), userContext); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(ref TKey key, long keyHash, TContext userContext = default) + public Status Delete(ReadOnlySpan key, long keyHash, TContext userContext = default) { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( - ref key, keyHash, userContext, sessionFunctions); + return clientSession.store.ContextDelete, TStoreFunctions, TAllocator>>( + key, keyHash, userContext, sessionFunctions); } /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, TContext userContext = default) - => Delete(ref key, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Status Delete(TKey key, ref DeleteOptions deleteOptions, TContext userContext = default) - => Delete(ref key, ref deleteOptions, userContext); - - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ResetModified(ref TKey key) - => clientSession.UnsafeResetModified(sessionFunctions, ref key); + public void ResetModified(ReadOnlySpan key) + => clientSession.UnsafeResetModified(sessionFunctions, key); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool IsModified(TKey key) - => clientSession.UnsafeIsModified(sessionFunctions, ref key); + internal bool IsModified(ReadOnlySpan key) + => clientSession.UnsafeIsModified(sessionFunctions, key); /// public void Refresh() { Debug.Assert(clientSession.store.epoch.ThisInstanceProtected()); - clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); + clientSession.store.InternalRefresh, TStoreFunctions, TAllocator>>(sessionFunctions); } #endregion ITsavoriteContext } diff --git a/libs/storage/Tsavorite/cs/src/core/Compaction/ICompactionFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Compaction/ICompactionFunctions.cs index 6e6f82d9677..29dad3630a8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Compaction/ICompactionFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Compaction/ICompactionFunctions.cs @@ -6,30 +6,24 @@ namespace Tsavorite.core /// /// Optional functions to be called during compaction. /// - /// - /// - public interface ICompactionFunctions + public interface ICompactionFunctions { /// /// Checks if record in the Tsavorite log is logically deleted. - /// If the record was deleted via - /// then this function is not called for such a record. + /// If the record was deleted the usual Delete() (i.e. its tombstone is set), then this function is not called for it. /// /// - /// - /// One possible scenario is if Tsavorite is used to store reference counted records. - /// Once the record count reaches zero it can be considered to be no longer relevant and - /// compaction can skip the record. - /// + /// One possible scenario is if Tsavorite is used to store reference counted records. If the refcount reaches zero + /// it can be considered to be no longer relevant and compaction can skip the record. /// - /// - /// - /// - bool IsDeleted(ref TKey key, ref TValue value); + bool IsDeleted(in TSourceLogRecord logRecord) + where TSourceLogRecord : ISourceLogRecord; } - internal struct DefaultCompactionFunctions : ICompactionFunctions + internal struct DefaultCompactionFunctions : ICompactionFunctions { - public bool IsDeleted(ref TKey key, ref TValue value) => false; + public bool IsDeleted(in TSourceLogRecord logRecord) + where TSourceLogRecord : ISourceLogRecord + => false; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Compaction/LogCompactionFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Compaction/LogCompactionFunctions.cs deleted file mode 100644 index 4c7e79f8cfb..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Compaction/LogCompactionFunctions.cs +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - internal sealed class LogCompactionFunctions : ISessionFunctions - where TFunctions : ISessionFunctions - { - readonly TFunctions _functions; - - public LogCompactionFunctions(TFunctions functions) - { - _functions = functions; - } - - /// - /// No reads during compaction - /// - public bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) => true; - - public bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - - public void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo) { } - - /// - /// No ConcurrentDeleter needed for compaction - /// - public bool ConcurrentDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - - /// - /// For compaction, we never perform concurrent writes as rolled over data defers to - /// newly inserted data for the same key. - /// - public bool ConcurrentWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => true; - - public bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - - public bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo) => true; - - public bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - public void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo) { } - - public bool InPlaceUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - - public bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; - - public bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo) => true; - - public void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } - - public void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } - - public int GetRMWModifiedValueLength(ref TValue value, ref TInput input) => 0; - public int GetRMWInitialValueLength(ref TInput input) => 0; - public int GetUpsertValueLength(ref TValue value, ref TInput input) => _functions.GetUpsertValueLength(ref value, ref input); - - /// - /// No reads during compaction - /// - public bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo) => true; - - /// - /// Write compacted live value to store - /// - public bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - => _functions.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); - - public void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason) { } - - public void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Compaction/TsavoriteCompaction.cs b/libs/storage/Tsavorite/cs/src/core/Compaction/TsavoriteCompaction.cs index 4bef9caa879..1c443476d92 100644 --- a/libs/storage/Tsavorite/cs/src/core/Compaction/TsavoriteCompaction.cs +++ b/libs/storage/Tsavorite/cs/src/core/Compaction/TsavoriteCompaction.cs @@ -6,58 +6,52 @@ namespace Tsavorite.core /// /// Compaction methods /// - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate /// - /// Functions used to manage key-values during compaction - /// User provided compaction functions (see ). - /// Input for SingleWriter - /// Output from SingleWriter; it will be called all records that are moved, before Compact() returns, so the user must supply buffering or process each output completely + /// User provided compaction functions (see ). /// Compact log until this address /// Compaction type (whether we lookup records or scan log for liveness checking) /// Address until which compaction was done - internal long Compact(TFunctions functions, TCompactionFunctions cf, ref TInput input, ref TOutput output, long untilAddress, CompactionType compactionType) - where TFunctions : ISessionFunctions - where TCompactionFunctions : ICompactionFunctions + internal long Compact(TCompactionFunctions cf, long untilAddress, CompactionType compactionType) + where TCompactionFunctions : ICompactionFunctions { return compactionType switch { - CompactionType.Scan => CompactScan(functions, cf, ref input, ref output, untilAddress), - CompactionType.Lookup => CompactLookup(functions, cf, ref input, ref output, untilAddress), + CompactionType.Scan => CompactScan(cf, untilAddress), + CompactionType.Lookup => CompactLookup(cf, untilAddress), _ => throw new TsavoriteException("Invalid compaction type"), }; } - private long CompactLookup(TFunctions functions, TCompactionFunctions cf, ref TInput input, ref TOutput output, long untilAddress) - where TFunctions : ISessionFunctions - where TCompactionFunctions : ICompactionFunctions + private long CompactLookup(TCompactionFunctions cf, long untilAddress) + where TCompactionFunctions : ICompactionFunctions { if (untilAddress > hlogBase.SafeReadOnlyAddress) throw new TsavoriteException("Can compact only until Log.SafeReadOnlyAddress"); - var lf = new LogCompactionFunctions(functions); - using var storeSession = NewSession>(lf); + using var storeSession = NewSession>(new()); var storebContext = storeSession.BasicContext; using (var iter1 = Log.Scan(Log.BeginAddress, untilAddress)) { long numPending = 0; - while (iter1.GetNext(out var recordInfo)) + while (iter1.GetNext()) { - ref var key = ref iter1.GetKey(); - ref var value = ref iter1.GetValue(); + var key = iter1.Key; - if (!recordInfo.Tombstone && !cf.IsDeleted(ref key, ref value)) + if (!iter1.Info.Tombstone && !cf.IsDeleted(in iter1)) { - var status = storebContext.CompactionCopyToTail(ref key, ref input, ref value, ref output, iter1.CurrentAddress, iter1.NextAddress); + var iter1AsLogSource = iter1 as ISourceLogRecord; // Can't use 'ref' on a 'using' variable + var status = storebContext.CompactionCopyToTail(ref iter1AsLogSource, iter1.CurrentAddress, iter1.NextAddress); if (status.IsPending && ++numPending > 256) { - storebContext.CompletePending(wait: true); + _ = storebContext.CompletePending(wait: true); numPending = 0; } } @@ -66,47 +60,45 @@ private long CompactLookup 0) - storebContext.CompletePending(wait: true); + _ = storebContext.CompletePending(wait: true); } Log.ShiftBeginAddress(untilAddress, false); return untilAddress; } - private long CompactScan(TFunctions functions, TCompactionFunctions cf, ref TInput input, ref TOutput output, long untilAddress) - where TFunctions : ISessionFunctions - where TCompactionFunctions : ICompactionFunctions + private long CompactScan(TCompactionFunctions cf, long untilAddress) + where TCompactionFunctions : ICompactionFunctions { if (untilAddress > hlogBase.SafeReadOnlyAddress) throw new TsavoriteException("Can compact only until Log.SafeReadOnlyAddress"); var originalUntilAddress = untilAddress; - var lf = new LogCompactionFunctions(functions); - using var storeSession = NewSession>(lf); + using var storeSession = NewSession>(new()); var storebContext = storeSession.BasicContext; - var tempKVSettings = new KVSettings(baseDir: null, loggerFactory: loggerFactory) + var tempKVSettings = new KVSettings(baseDir: null, loggerFactory: loggerFactory) { - IndexSize = KVSettings.SetIndexSizeFromCacheLines(IndexSize), + IndexSize = KVSettings.SetIndexSizeFromCacheLines(IndexSize), LogDevice = new NullDevice(), ObjectLogDevice = new NullDevice() }; - using (var tempKv = new TsavoriteKV(tempKVSettings, storeFunctions, allocatorFactory)) - using (var tempKvSession = tempKv.NewSession(functions)) + using (var tempKv = new TsavoriteKV(tempKVSettings, storeFunctions, allocatorFactory)) + using (var tempKvSession = tempKv.NewSession>(new())) { var tempbContext = tempKvSession.BasicContext; using (var iter1 = Log.Scan(hlogBase.BeginAddress, untilAddress)) { - while (iter1.GetNext(out var recordInfo)) + while (iter1.GetNext()) { - ref var key = ref iter1.GetKey(); - ref var value = ref iter1.GetValue(); - - if (recordInfo.Tombstone || cf.IsDeleted(ref key, ref value)) - tempbContext.Delete(ref key); + if (iter1.Info.Tombstone || cf.IsDeleted(in iter1)) + _ = tempbContext.Delete(iter1.Key); else - tempbContext.Upsert(ref key, ref value); + { + var iterLogRecord = iter1 as ISourceLogRecord; // Can't use 'ref' on a 'using' variable + _ = tempbContext.Upsert(ref iterLogRecord); + } } // Ensure address is at record boundary untilAddress = originalUntilAddress = iter1.NextAddress; @@ -119,9 +111,9 @@ private long CompactScan= untilAddress (scan boundary), we are safe to copy the old record // to the tail. We don't know the actualAddress of the key in the main kv, but we it will not be below untilAddress. - var status = storebContext.CompactionCopyToTail(ref iter3.GetKey(), ref input, ref iter3.GetValue(), ref output, iter3.CurrentAddress, untilAddress - 1); + var iter3AsLogSource = iter3 as ISourceLogRecord; // Can't use 'ref' on a 'using' variable + var status = storebContext.CompactionCopyToTail(ref iter3AsLogSource, iter3.CurrentAddress, untilAddress - 1); if (status.IsPending && ++numPending > 256) { - storebContext.CompletePending(wait: true); + _ = storebContext.CompletePending(wait: true); numPending = 0; } } if (numPending > 0) - storebContext.CompletePending(wait: true); + _ = storebContext.CompletePending(wait: true); } Log.ShiftBeginAddress(originalUntilAddress, false); return originalUntilAddress; } private void ScanImmutableTailToRemoveFromTempKv(ref long untilAddress, long scanUntil, - BasicContext tempbContext) - where TFunctions : ISessionFunctions + BasicContext tempbContext) + where TFunctions : ISessionFunctions { using var iter = Log.Scan(untilAddress, scanUntil); - while (iter.GetNext(out var _)) + while (iter.GetNext()) { - tempbContext.Delete(ref iter.GetKey(), default); + _ = tempbContext.Delete(iter.Key, default); untilAddress = iter.NextAddress; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/ICheckpointNamingScheme.cs b/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/ICheckpointNamingScheme.cs index a0fa9d36f2f..5ce8ebf7892 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/ICheckpointNamingScheme.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/ICheckpointNamingScheme.cs @@ -81,7 +81,6 @@ public interface ICheckpointNamingScheme string LogCheckpointBasePath { get; } /// - /// Get base path holding TsavoriteLog commits /// string TsavoriteLogCommitBasePath { get; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 1295be24618..d2cc3843190 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -8,21 +8,21 @@ namespace Tsavorite.core public static class Checkpoint { #region Single-store APIs - public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { guid = Guid.NewGuid(); - var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); + var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); if (checkpointType == CheckpointType.FoldOver) { - var backend = new FoldOverSMTask(store, guid); + var backend = new FoldOverSMTask(store, guid); return new FullCheckpointSM(indexCheckpointTask, backend); } else if (checkpointType == CheckpointType.Snapshot) { - var backend = new SnapshotCheckpointSMTask(store, guid); + var backend = new SnapshotCheckpointSMTask(store, guid); return new FullCheckpointSM(indexCheckpointTask, backend); } else @@ -31,38 +31,38 @@ public static IStateMachine Full(Tsav } } - public static IStateMachine Streaming(TsavoriteKV store, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine Streaming(TsavoriteKV store, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { guid = Guid.NewGuid(); - var backend = new StreamingSnapshotCheckpointSMTask(store, guid); + var backend = new StreamingSnapshotCheckpointSMTask(store, guid); return new StreamingSnapshotCheckpointSM(backend); } - public static IStateMachine IndexOnly(TsavoriteKV store, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine IndexOnly(TsavoriteKV store, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { guid = Guid.NewGuid(); - var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); + var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); return new IndexCheckpointSM(indexCheckpointTask); } - public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { guid = Guid.NewGuid(); if (checkpointType == CheckpointType.FoldOver) { - var backend = new FoldOverSMTask(store, guid); + var backend = new FoldOverSMTask(store, guid); return new HybridLogCheckpointSM(backend); } else if (checkpointType == CheckpointType.Snapshot) { - var backend = new SnapshotCheckpointSMTask(store, guid); + var backend = new SnapshotCheckpointSMTask(store, guid); return new HybridLogCheckpointSM(backend); } else @@ -71,39 +71,39 @@ public static IStateMachine HybridLogOnly(TsavoriteKV store, Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine IncrementalHybridLogOnly(TsavoriteKV store, Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); + var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); return new HybridLogCheckpointSM(backend); } #endregion #region Two-store APIs - public static IStateMachine Full( - TsavoriteKV store1, - TsavoriteKV store2, + public static IStateMachine Full( + TsavoriteKV store1, + TsavoriteKV store2, CheckpointType checkpointType, out Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); - var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); + var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); if (checkpointType == CheckpointType.FoldOver) { - var backend1 = new FoldOverSMTask(store1, guid); - var backend2 = new FoldOverSMTask(store2, guid); + var backend1 = new FoldOverSMTask(store1, guid); + var backend2 = new FoldOverSMTask(store2, guid); return new FullCheckpointSM(indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); } else if (checkpointType == CheckpointType.Snapshot) { - var backend1 = new SnapshotCheckpointSMTask(store1, guid); - var backend2 = new SnapshotCheckpointSMTask(store2, guid); + var backend1 = new SnapshotCheckpointSMTask(store1, guid); + var backend2 = new SnapshotCheckpointSMTask(store2, guid); return new FullCheckpointSM(indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); } else @@ -112,57 +112,57 @@ public static IStateMachine Full( - TsavoriteKV store1, - TsavoriteKV store2, + public static IStateMachine Streaming( + TsavoriteKV store1, + TsavoriteKV store2, out Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var backend1 = new StreamingSnapshotCheckpointSMTask(store1, guid); - var backend2 = new StreamingSnapshotCheckpointSMTask(store2, guid); + var backend1 = new StreamingSnapshotCheckpointSMTask(store1, guid); + var backend2 = new StreamingSnapshotCheckpointSMTask(store2, guid); return new StreamingSnapshotCheckpointSM(backend1, backend2); } - public static IStateMachine IndexOnly( - TsavoriteKV store1, - TsavoriteKV store2, + public static IStateMachine IndexOnly( + TsavoriteKV store1, + TsavoriteKV store2, out Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); - var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); + var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); return new IndexCheckpointSM(indexCheckpointTask1, indexCheckpointTask2); } - public static IStateMachine HybridLogOnly( - TsavoriteKV store1, - TsavoriteKV store2, + public static IStateMachine HybridLogOnly( + TsavoriteKV store1, + TsavoriteKV store2, CheckpointType checkpointType, out Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { guid = Guid.NewGuid(); if (checkpointType == CheckpointType.FoldOver) { - var backend1 = new FoldOverSMTask(store1, guid); - var backend2 = new FoldOverSMTask(store2, guid); + var backend1 = new FoldOverSMTask(store1, guid); + var backend2 = new FoldOverSMTask(store2, guid); return new HybridLogCheckpointSM(backend1, backend2); } else if (checkpointType == CheckpointType.Snapshot) { - var backend1 = new SnapshotCheckpointSMTask(store1, guid); - var backend2 = new SnapshotCheckpointSMTask(store2, guid); + var backend1 = new SnapshotCheckpointSMTask(store1, guid); + var backend2 = new SnapshotCheckpointSMTask(store2, guid); return new HybridLogCheckpointSM(backend1, backend2); } else @@ -171,17 +171,17 @@ public static IStateMachine HybridLogOnly( - TsavoriteKV store1, - TsavoriteKV store2, + public static IStateMachine IncrementalHybridLogOnly( + TsavoriteKV store1, + TsavoriteKV store2, Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { - var backend1 = new IncrementalSnapshotCheckpointSMTask(store1, guid); - var backend2 = new IncrementalSnapshotCheckpointSMTask(store2, guid); + var backend1 = new IncrementalSnapshotCheckpointSMTask(store1, guid); + var backend2 = new IncrementalSnapshotCheckpointSMTask(store2, guid); return new HybridLogCheckpointSM(backend1, backend2); } #endregion diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index 348da3ece53..61effafc039 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -10,11 +10,11 @@ namespace Tsavorite.core /// version on the log and waiting until it is flushed to disk. It is simple and fast, but can result /// in garbage entries on the log, and a slower recovery of performance. /// - internal sealed class FoldOverSMTask : HybridLogCheckpointSMTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class FoldOverSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public FoldOverSMTask(TsavoriteKV store, Guid guid) + public FoldOverSMTask(TsavoriteKV store, Guid guid) : base(store, guid) { } 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 dc2e4f6eb2b..0ccd9848c24 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -11,16 +11,16 @@ namespace Tsavorite.core /// This task is the base class for a checkpoint "backend", which decides how a captured version is /// persisted on disk. /// - internal abstract class HybridLogCheckpointSMTask : IStateMachineTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal abstract class HybridLogCheckpointSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - protected readonly TsavoriteKV store; + protected readonly TsavoriteKV store; protected long lastVersion; protected readonly Guid guid; protected bool isStreaming; - public HybridLogCheckpointSMTask(TsavoriteKV store, Guid guid) + public HybridLogCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; this.guid = guid; @@ -73,7 +73,7 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv } } - protected static void CollectMetadata(SystemState next, TsavoriteKV store) + protected static void CollectMetadata(SystemState next, TsavoriteKV store) { // Collect object log offsets only after flushes // are completed diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs index 22d7dffe6c7..09f9307e335 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs @@ -10,11 +10,11 @@ namespace Tsavorite.core /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place /// update performance as it does not advance the readonly marker unnecessarily. /// - internal sealed class IncrementalSnapshotCheckpointSMTask : HybridLogCheckpointSMTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class IncrementalSnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public IncrementalSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) + public IncrementalSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) : base(store, guid) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs index 33c1217dbf5..18f889102ae 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs @@ -9,14 +9,14 @@ namespace Tsavorite.core /// /// This task performs an index checkpoint. /// - internal sealed class IndexCheckpointSMTask : IStateMachineTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class IndexCheckpointSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - readonly TsavoriteKV store; + readonly TsavoriteKV store; readonly Guid guid; - public IndexCheckpointSMTask(TsavoriteKV store, Guid guid) + public IndexCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; this.guid = guid; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs index 986f8e09a7b..c96a72876d3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs @@ -8,14 +8,14 @@ namespace Tsavorite.core /// /// Resizes an index /// - internal sealed class IndexResizeSMTask : IStateMachineTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class IndexResizeSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - readonly TsavoriteKV store; + readonly TsavoriteKV store; long lastVersion; - public IndexResizeSMTask(TsavoriteKV store) + public IndexResizeSMTask(TsavoriteKV store) { this.store = store; } 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 3ea849d4978..743471b56bf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -10,11 +10,11 @@ namespace Tsavorite.core /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place /// update performance as it does not advance the readonly marker unnecessarily. /// - internal sealed class SnapshotCheckpointSMTask : HybridLogCheckpointSMTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class SnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public SnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) + public SnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) : base(store, guid) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index bdaed75e42f..b29ba5e5c5e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -13,11 +13,11 @@ namespace Tsavorite.core /// it does not require a snapshot of the index. Recovery is achieved by replaying the yielded log /// of key-value pairs and inserting each record into an empty database. /// - sealed class StreamingSnapshotCheckpointSMTask : HybridLogCheckpointSMTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + sealed class StreamingSnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public StreamingSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) + public StreamingSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) : base(store, guid) { isStreaming = true; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs index a6dcda90a8d..7c3e800094b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs @@ -6,28 +6,28 @@ namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; + IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; long scannedUntilAddressCursor; long numberOfRecords; - class StreamingSnapshotSessionFunctions : SessionFunctionsBase + class StreamingSnapshotSessionFunctions : SessionFunctionsBase { } - class ScanPhase1Functions : IScanIteratorFunctions + class ScanPhase1Functions : IScanIteratorFunctions { - readonly IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; + readonly IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; readonly Guid checkpointToken; readonly long currentVersion; readonly long nextVersion; public long numberOfRecords; - public ScanPhase1Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, Guid checkpointToken, long currentVersion, long nextVersion) + public ScanPhase1Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, Guid checkpointToken, long currentVersion, long nextVersion) { this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; this.checkpointToken = checkpointToken; @@ -36,16 +36,13 @@ public ScanPhase1Functions(IStreamingSnapshotIteratorFunctions str } /// - public bool SingleReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; - return streamingSnapshotIteratorFunctions.Reader(ref key, ref value, recordMetadata, numberOfRecords); + return streamingSnapshotIteratorFunctions.Reader(ref srcLogRecord, recordMetadata, numberOfRecords); } - /// - public bool ConcurrentReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - /// public void OnException(Exception exception, long numberOfRecords) => streamingSnapshotIteratorFunctions.OnException(exception, numberOfRecords); @@ -80,28 +77,25 @@ internal void StreamingSnapshotScanPhase1() } } - class ScanPhase2Functions : IScanIteratorFunctions + class ScanPhase2Functions : IScanIteratorFunctions { - readonly IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; + readonly IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; readonly long phase1NumberOfRecords; - public ScanPhase2Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, long acceptedRecordCount) + public ScanPhase2Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, long acceptedRecordCount) { this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; this.phase1NumberOfRecords = acceptedRecordCount; } /// - public bool SingleReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + where TSourceLogRecord : ISourceLogRecord { cursorRecordResult = CursorRecordResult.Accept; - return streamingSnapshotIteratorFunctions.Reader(ref key, ref value, recordMetadata, numberOfRecords); + return streamingSnapshotIteratorFunctions.Reader(ref srcLogRecord, recordMetadata, numberOfRecords); } - /// - public bool ConcurrentReader(ref TKey key, ref TValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - /// public void OnException(Exception exception, long numberOfRecords) => streamingSnapshotIteratorFunctions.OnException(exception, numberOfRecords); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs index f4f941027a8..90fa8afa6b0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs @@ -3,9 +3,9 @@ namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal long lastVersion; 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 9bdbb4e9562..9700d905282 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/CompletedOutput.cs @@ -6,28 +6,29 @@ namespace Tsavorite.core { /// - /// A list of for completed outputs from a pending operation. + /// A list of for completed outputs from a pending operation. /// /// The session holds this list and returns an enumeration to the caller of an appropriate CompletePending overload. The session will handle /// disposing and clearing this list, but it is best if the caller calls Dispose() after processing the results, so the key, input, and heap containers /// are released as soon as possible. - public sealed class CompletedOutputIterator : IDisposable + public sealed class CompletedOutputIterator : IDisposable { internal const int kInitialAlloc = 32; internal const int kReallocMultuple = 2; - internal CompletedOutput[] vector = new CompletedOutput[kInitialAlloc]; + internal CompletedOutput[] vector = new CompletedOutput[kInitialAlloc]; internal int maxIndex = -1; internal int currentIndex = -1; - internal void TransferFrom(ref TsavoriteKV.PendingContext pendingContext, Status status) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal void TransferFrom(ref TsavoriteKV.PendingContext pendingContext, + Status status, SectorAlignedBufferPool bufferPool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Note: vector is never null if (maxIndex >= vector.Length - 1) Array.Resize(ref vector, vector.Length * kReallocMultuple); ++maxIndex; - vector[maxIndex].TransferFrom(ref pendingContext, status); + vector[maxIndex].TransferFrom(ref pendingContext, status, bufferPool); } /// @@ -52,7 +53,7 @@ public bool Next() /// if there is no current element, either because Next() has not been called or it has advanced /// past the last element of the array /// - public ref CompletedOutput Current => ref vector[currentIndex]; + public ref CompletedOutput Current => ref vector[currentIndex]; /// public void Dispose() @@ -69,15 +70,15 @@ public void Dispose() /// The session holds a list of these that it returns to the caller of an appropriate CompletePending overload. The session will handle disposing /// and clearing, and will manage Dispose(), but it is best if the caller calls Dispose() after processing the results, so the key, input, and heap containers /// are released as soon as possible. - public struct CompletedOutput + public struct CompletedOutput { - private IHeapContainer keyContainer; + private SpanByteHeapContainer keyContainer; private IHeapContainer inputContainer; /// /// The key for this pending operation. /// - public ref TKey Key => ref keyContainer.Get(); + public ReadOnlySpan Key => keyContainer.Get().ReadOnlySpan; /// /// The input for this pending operation. @@ -104,19 +105,19 @@ public struct CompletedOutput /// public Status Status; - internal void TransferFrom(ref TsavoriteKV.PendingContext pendingContext, Status status) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal void TransferFrom(ref TsavoriteKV.PendingContext pendingContext, + Status status, SectorAlignedBufferPool bufferPool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Transfers the containers from the pendingContext, then null them; this is called before pendingContext.Dispose(). - keyContainer = pendingContext.key; - pendingContext.key = null; + keyContainer = new SpanByteHeapContainer(pendingContext.Key, bufferPool); inputContainer = pendingContext.input; pendingContext.input = null; Output = pendingContext.output; Context = pendingContext.userContext; - RecordMetadata = new(pendingContext.recordInfo, pendingContext.logicalAddress); + RecordMetadata = new(pendingContext.logicalAddress); Status = status; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index 8853ca01d88..d6f046d0aa0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -8,9 +8,9 @@ namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal sealed class TsavoriteExecutionContext { @@ -27,19 +27,19 @@ internal sealed class TsavoriteExecutionContext public long totalPending; public readonly Dictionary> ioPendingRequests; public readonly AsyncCountDown pendingReads; - public readonly AsyncQueue> readyResponses; + public readonly AsyncQueue readyResponses; public int asyncPendingCount; internal RevivificationStats RevivificationStats = new(); - public bool isAcquiredLockable; + public bool isAcquiredTransactional; public TsavoriteExecutionContext(int sessionID) { SessionState = SystemState.Make(Phase.REST, 1); this.sessionID = sessionID; - readyResponses = new AsyncQueue>(); + readyResponses = new AsyncQueue(); ioPendingRequests = new Dictionary>(); pendingReads = new AsyncCountDown(); - isAcquiredLockable = false; + isAcquiredTransactional = false; } public int SyncIoPendingCount => ioPendingRequests.Count - asyncPendingCount; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs index bfb6d9468a1..98a92c22d2e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs @@ -10,7 +10,7 @@ namespace Tsavorite.core /// /// Configuration settings for hybrid log. Use Utility.ParseSize to specify sizes in familiar string notation (e.g., "4k" and "4 MB"). /// - public sealed class KVSettings : IDisposable + public sealed class KVSettings : IDisposable { readonly bool disposeDevices = false; readonly bool deleteDirOnDispose = false; @@ -82,9 +82,7 @@ public sealed class KVSettings : IDisposable public long ReadCacheMemorySize = 1L << 34; /// - /// Fraction of log head (in memory) used for second chance - /// copy to tail. This is (1 - MutableFraction) for the - /// underlying log. + /// Fraction of log head (in memory) used for second chance copy to tail. This is (1 - MutableFraction) for the underlying log. /// public double ReadCacheSecondChanceFraction = 0.1; @@ -129,6 +127,16 @@ public sealed class KVSettings : IDisposable /// public StateMachineDriver StateMachineDriver = null; + /// + /// Maximum size of a key stored inline in the in-memory portion of the main log for both allocators. + /// + public int MaxInlineKeySize = 1 << LogSettings.kDefaultMaxInlineKeySizeBits; + + /// + /// Maximum size of a valuie stored inline in the in-memory portion of the main log for . + /// + public int MaxInlineValueSize = 1 << LogSettings.kDefaultMaxInlineValueSizeBits; + /// /// Create default configuration settings for TsavoriteKV. You need to create and specify LogDevice /// explicitly with this API. @@ -157,9 +165,6 @@ public KVSettings(string baseDir, bool deleteDirOnDispose = false, ILoggerFactor this.baseDir = baseDir; LogDevice = baseDir == null ? new NullDevice() : Devices.CreateLogDevice(baseDir + "/hlog.log", deleteOnClose: deleteDirOnDispose); - if (!Utility.IsBlittable() || !Utility.IsBlittable()) - ObjectLogDevice = baseDir == null ? new NullDevice() : Devices.CreateLogDevice(baseDir + "/hlog.obj.log", deleteOnClose: deleteDirOnDispose); - CheckpointDir = baseDir == null ? null : baseDir + "/checkpoints"; } @@ -217,7 +222,9 @@ internal LogSettings GetLogSettings() MutableFraction = MutableFraction, MinEmptyPageCount = MinEmptyPageCount, PreallocateLog = PreallocateLog, - ReadCacheSettings = GetReadCacheSettings() + ReadCacheSettings = GetReadCacheSettings(), + MaxInlineKeySizeBits = Utility.NumBitsPreviousPowerOf2(MaxInlineKeySize), + MaxInlineValueSizeBits = Utility.NumBitsPreviousPowerOf2(MaxInlineValueSize) }; } 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 2e5114e4208..8b2447f0e9b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSettings.cs @@ -25,6 +25,18 @@ internal class LogSettings /// Maximum number of bits for the size of the in-memory portion of the log public const int kMaxMemorySizeBits = kMaxSegmentSizeBits; + /// Default number of bits for the size of an inline (not overflow) key + public const int kDefaultMaxInlineKeySizeBits = kLowestMaxInlineSizeBits + 1; + + /// Default number of bits for the size of an inline (not overflow) value, for + public const int kDefaultMaxInlineValueSizeBits = 10; + + /// Minimum number of bits for the size of an overflow (int inline) key or value + public const int kLowestMaxInlineSizeBits = kMinPageSizeBits - 1; + + /// Maximum size of a string is 512MB + public const int kMaxStringSizeBits = 29; + /// /// Device used for main hybrid log /// @@ -50,6 +62,11 @@ internal class LogSettings /// public int MemorySizeBits = 34; + /// + /// Size of pages for ObjectAllocator key and optional fields space + /// + public int ObjectLogVariableSpacePageSizeBits = 25; + /// /// Controls how many pages should be empty to account for non-power-of-two-sized log /// @@ -75,5 +92,15 @@ internal class LogSettings /// Whether to preallocate the entire log (pages) in memory /// public bool PreallocateLog = false; + + /// + /// Maximum size of a key stored inline in the in-memory portion of the main log for both allocators. + /// + public int MaxInlineKeySizeBits = kDefaultMaxInlineKeySizeBits; + + /// + /// Maximum size of a value stored inline in the in-memory portion of the main log for . + /// + public int MaxInlineValueSizeBits = kDefaultMaxInlineValueSizeBits; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSizeTracker.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSizeTracker.cs index f1994158dbe..80875ffac52 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSizeTracker.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/LogSizeTracker.cs @@ -10,16 +10,12 @@ namespace Tsavorite.core { /// Interface for calculating the size of the log - /// Type of key - /// Type of value - public interface ILogSizeCalculator + public interface ILogSizeCalculator { /// Calculates the size of a log record - /// Information about the record - /// The key - /// The value + /// The record being evaluated /// The size of the record - long CalculateRecordSize(RecordInfo recordInfo, TKey key, TValue value); + long CalculateRecordSize(ref TSourceLogRecord logRecord) where TSourceLogRecord : ISourceLogRecord; } public enum LogOperationType @@ -27,15 +23,15 @@ public enum LogOperationType Deserialize } - public class LogOperationObserver : IObserver> - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - where TLogSizeCalculator : ILogSizeCalculator + public class LogOperationObserver : IObserver + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + where TLogSizeCalculator : ILogSizeCalculator { - private readonly LogSizeTracker logSizeTracker; + private readonly LogSizeTracker logSizeTracker; private readonly LogOperationType logOperationType; - public LogOperationObserver(LogSizeTracker logSizeTracker, LogOperationType logOperationType) + public LogOperationObserver(LogSizeTracker logSizeTracker, LogOperationType logOperationType) { this.logSizeTracker = logSizeTracker; this.logOperationType = logOperationType; @@ -45,37 +41,28 @@ public void OnCompleted() { } public void OnError(Exception error) { } - public void OnNext(ITsavoriteScanIterator records) + public void OnNext(ITsavoriteScanIterator records) { long size = 0; - while (records.GetNext(out RecordInfo info, out TKey key, out TValue value)) - { - Debug.Assert(key != null); - Debug.Assert(value != null); - - size += logSizeTracker.LogSizeCalculator.CalculateRecordSize(info, key, value); - } + while (records.GetNext()) + size += logSizeTracker.LogSizeCalculator.CalculateRecordSize(ref records); if (size == 0) return; if (logOperationType == LogOperationType.Deserialize) - { logSizeTracker.IncrementSize(size); - } } } /// Tracks and controls size of log - /// Type of key - /// Type of value /// /// /// Type of the log size calculator - public class LogSizeTracker : IObserver> - where TLogSizeCalculator : ILogSizeCalculator - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public class LogSizeTracker : IObserver + where TLogSizeCalculator : ILogSizeCalculator + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { private ConcurrentCounter logSize; private long lowTargetSize; @@ -84,7 +71,7 @@ public class LogSizeTracker logAccessor; + internal LogAccessor logAccessor; /// Indicates whether resizer task has been stopped public volatile bool Stopped; @@ -105,7 +92,7 @@ public class LogSizeTrackerTarget size for the hybrid log memory utilization /// Delta from target size to maintain memory utilization /// - public LogSizeTracker(LogAccessor logAccessor, TLogSizeCalculator logSizeCalculator, long targetSize, long delta, ILogger logger) + public LogSizeTracker(LogAccessor logAccessor, TLogSizeCalculator logSizeCalculator, long targetSize, long delta, ILogger logger) { Debug.Assert(logAccessor != null); Debug.Assert(logSizeCalculator != null); @@ -141,18 +128,14 @@ public void OnCompleted() { } public void OnError(Exception error) { } /// Callback on allocator evicting a page to disk - public void OnNext(ITsavoriteScanIterator records) + public void OnNext(ITsavoriteScanIterator records) { long size = 0; - while (records.GetNext(out RecordInfo info, out TKey key, out TValue value)) - { - Debug.Assert(key != null); - Debug.Assert(value != null); - - size += LogSizeCalculator.CalculateRecordSize(info, key, value); - } + while (records.GetNext()) + size += LogSizeCalculator.CalculateRecordSize(ref records); - if (size == 0) return; + if (size == 0) + return; IncrementSize(-size); // Reduce size as records are being evicted } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationOptions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationOptions.cs index 7482f377d2f..bc03960ddfc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationOptions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationOptions.cs @@ -8,7 +8,7 @@ namespace Tsavorite.core /// /// Identifies which log regions records will be copied from to . This specification is /// evaluated in hierarchical order, from that on the TsavoriteKV ctor, which may be overridden by those in - /// .NewSession(), which may be overridden + /// .NewSession(), which may be overridden /// by those at the individual Read() level. /// public enum ReadCopyFrom : byte diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs index 5a15ea78d6e..486218300ad 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs @@ -58,8 +58,8 @@ internal enum OperationStatus RETRY_LATER, /// - /// I/O has been enqueued and the caller must go through or - /// , + /// I/O has been enqueued and the caller must go through or + /// , /// or one of the Async forms. /// RECORD_ON_DISK, @@ -72,7 +72,7 @@ internal enum OperationStatus /// /// Allocation failed, due to a need to flush pages. Clients do not see this status directly; they see . /// - /// For Sync operations we retry this as part of . + /// For Sync operations we retry this as part of . /// For Async operations we retry this as part of the ".Complete(...)" or ".CompleteAsync(...)" operation on the appropriate "*AsyncResult{}" object. /// /// 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 bcb10637121..2d9e6c9da40 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/PendingContext.cs @@ -1,20 +1,36 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; +using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - internal struct PendingContext + internal unsafe struct PendingContext : ISourceLogRecord { // User provided information internal OperationType type; - internal IHeapContainer key; - internal IHeapContainer value; + + /// + /// DiskLogRecord carries either the input to RUMD operations or 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 + /// For pending ConditionalCopy operations, where it is one of: + /// + /// A created by serializing from an in-memory + /// A retrieved from the disk, for operations such as Compact + /// + /// + /// + /// + internal DiskLogRecord diskLogRecord; + internal IHeapContainer input; internal TOutput output; internal TContext userContext; @@ -23,19 +39,16 @@ internal struct PendingContext // Some additional information about the previous attempt internal long id; internal long logicalAddress; - internal long InitialLatestLogicalAddress; + internal long initialLatestLogicalAddress; // operationFlags values internal ushort operationFlags; internal const ushort kNoOpFlags = 0; - internal const ushort kNoKey = 0x0001; - internal const ushort kIsAsync = 0x0002; - internal const ushort kIsReadAtAddress = 0x0004; + internal const ushort kIsNoKey = 0x0001; + internal const ushort kIsReadAtAddress = 0x0002; internal ReadCopyOptions readCopyOptions; // Two byte enums - internal WriteReason writeReason; // for ConditionalCopyToTail; one byte enum - internal RecordInfo recordInfo; internal long minAddress; internal long maxAddress; @@ -45,63 +58,190 @@ internal struct PendingContext // For RMW if an allocation caused the source record for a copy to go from readonly to below HeadAddress, or for any operation with CAS failure. internal long retryNewLogicalAddress; - internal ScanCursorState scanCursorState; + // Address of the initial entry in the hash chain upon start of Internal(RUMD). + internal long initialEntryAddress; + + internal ScanCursorState scanCursorState; [MethodImpl(MethodImplOptions.AggressiveInlining)] internal PendingContext(long keyHash) => this.keyHash = keyHash; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal PendingContext(ReadCopyOptions sessionReadCopyOptions, ref ReadOptions readOptions, bool isAsync = false, bool noKey = false) + internal PendingContext(ReadCopyOptions sessionReadCopyOptions, ref ReadOptions readOptions) { // The async flag is often set when the PendingContext is created, so preserve that. - operationFlags = (ushort)((noKey ? kNoKey : kNoOpFlags) | (isAsync ? kIsAsync : kNoOpFlags)); + operationFlags = kNoOpFlags; readCopyOptions = ReadCopyOptions.Merge(sessionReadCopyOptions, readOptions.CopyOptions); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal PendingContext(ReadCopyOptions readCopyOptions, bool isAsync = false, bool noKey = false) + internal PendingContext(ReadCopyOptions readCopyOptions) { // The async flag is often set when the PendingContext is created, so preserve that. - operationFlags = (ushort)((noKey ? kNoKey : kNoOpFlags) | (isAsync ? kIsAsync : kNoOpFlags)); + operationFlags = kNoOpFlags; this.readCopyOptions = readCopyOptions; } - internal bool NoKey + internal readonly bool IsNoKey => (operationFlags & kIsNoKey) != 0; + internal void SetIsNoKey() => operationFlags |= kIsNoKey; + + internal readonly bool HasMinAddress => minAddress != Constants.kInvalidAddress; + + internal readonly bool IsReadAtAddress => (operationFlags & kIsReadAtAddress) != 0; + internal void SetIsReadAtAddress() => operationFlags |= kIsReadAtAddress; + + public void Dispose() { - readonly get => (operationFlags & kNoKey) != 0; - set => operationFlags = value ? (ushort)(operationFlags | kNoKey) : (ushort)(operationFlags & ~kNoKey); + diskLogRecord.Dispose(); + diskLogRecord = default; + input?.Dispose(); + input = default; } - internal readonly bool HasMinAddress => minAddress != Constants.kInvalidAddress; + #region Serialized Record Creation + /// + /// Serialize for Read and RMW operations; no Value is passed + /// + /// Record key + /// Input to the operation + /// Output from the operation + /// User context for the operation + /// Session functions wrapper for the operation + /// Allocator for backing storage + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void SerializeForReadOrRMW(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext, + TSessionFunctionsWrapper sessionFunctions, SectorAlignedBufferPool bufferPool) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + { + if (diskLogRecord.IsSet) + return; + diskLogRecord.SerializeForPendingReadOrRMW(key, bufferPool); + CopyIOC(ref input, output, userContext, sessionFunctions); + } - internal bool IsAsync + /// + /// 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(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, TContext userContext, TSessionFunctionsWrapper sessionFunctions, + SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - readonly get => (operationFlags & kIsAsync) != 0; - set => operationFlags = value ? (ushort)(operationFlags | kIsAsync) : (ushort)(operationFlags & ~kIsAsync); + Serialize(ref srcLogRecord, bufferPool, valueSerializer); + CopyIOC(ref input, output, userContext, sessionFunctions); } - internal bool IsReadAtAddress + /// + /// Serialize a or into the local for Pending operations + /// + /// The log record to be copied into the . This may be either in-memory or from disk IO + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void Serialize(ref TSourceLogRecord srcLogRecord, SectorAlignedBufferPool bufferPool, IObjectSerializer valueSerializer) + where TSourceLogRecord : ISourceLogRecord { - readonly get => (operationFlags & kIsReadAtAddress) != 0; - set => operationFlags = value ? (ushort)(operationFlags | kIsReadAtAddress) : (ushort)(operationFlags & ~kIsReadAtAddress); + Debug.Assert(!diskLogRecord.IsSet, "Should not try to reset PendingContext.diskLogRecord"); + if (srcLogRecord.AsLogRecord(out var logRecord)) + { + diskLogRecord.Serialize(in logRecord, bufferPool, valueSerializer); + 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); } - // RecordInfo is not used as such during the pending phase, so we reuse the space here. - internal long InitialEntryAddress + private void CopyIOC(ref TInput input, TOutput output, TContext userContext, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - readonly get => recordInfo.PreviousAddress; - set => recordInfo.PreviousAddress = value; + if (this.input == default) + this.input = new StandardHeapContainer(ref input); + this.output = output; + sessionFunctions.ConvertOutputToHeap(ref input, ref this.output); + this.userContext = userContext; } - public void Dispose() + #endregion // Serialized Record Creation + + #region ISourceLogRecord + /// + public readonly ref RecordInfo InfoRef => ref diskLogRecord.InfoRef; + /// + public readonly RecordInfo Info => diskLogRecord.Info; + + /// + public readonly bool IsSet => diskLogRecord.IsSet; + + /// + public readonly ReadOnlySpan Key => diskLogRecord.Key; + + /// + public readonly bool IsPinnedKey => diskLogRecord.IsPinnedKey; + + /// + public byte* PinnedKeyPointer => diskLogRecord.PinnedKeyPointer; + + /// + public readonly unsafe Span ValueSpan => diskLogRecord.ValueSpan; + + /// + public readonly IHeapObject ValueObject => diskLogRecord.ValueObject; + + /// + public ReadOnlySpan RecordSpan => diskLogRecord.RecordSpan; + + /// + public bool IsPinnedValue => diskLogRecord.IsPinnedValue; + + /// + public byte* PinnedValuePointer => diskLogRecord.PinnedValuePointer; + + /// + public readonly long ETag => diskLogRecord.ETag; + + /// + public readonly long Expiration => diskLogRecord.Expiration; + + /// + public readonly void ClearValueObject(Action disposer) { } // Not relevant for PendingContext + + /// + public readonly bool AsLogRecord(out LogRecord logRecord) { - key?.Dispose(); - key = default; - value?.Dispose(); - value = default; - input?.Dispose(); - input = default; + logRecord = default; + return false; } + + /// + public readonly bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) + { + diskLogRecord = this.diskLogRecord; + return true; + } + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly RecordFieldInfo GetRecordFieldInfo() => new() + { + KeyDataSize = Key.Length, + ValueDataSize = Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : ValueSpan.Length, + ValueIsObject = Info.ValueIsObject, + HasETag = Info.HasETag, + HasExpiration = Info.HasExpiration + }; + #endregion // ISourceLogRecord } } } \ No newline at end of file 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 5d82c473f53..ea7e08521c6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordInfo.cs @@ -10,42 +10,58 @@ namespace Tsavorite.core { - // RecordInfo layout (64 bits total): - // [Unused1][Modified][InNewVersion][Filler][Dirty][ETag][Sealed][Valid][Tombstone][LLLLLLL] [RAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] - // where L = leftover, R = readcache, A = address + // RecordInfo layout (64 bits total, high to low): + // [Unused1][Modified][InNewVersion][Filler][Dirty][Unused2][Sealed][Valid][Tombstone] + // [PreviousAddressIsOnDisk][HasExpiration][HasETag][Unused4][ValueIsObject][ValueIsInline][KeyIsInline] + // [RAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] [AAAAAAAA] where R = readcache, A = address [StructLayout(LayoutKind.Explicit, Size = 8)] public struct RecordInfo { +#pragma warning disable IDE1006 // Naming Styles: Must begin with uppercase letter const int kTotalSizeInBytes = 8; const int kTotalBits = kTotalSizeInBytes * 8; - // Previous address + // Previous address is the low 48 bits, with the 48th being the readcache bit internal const int kPreviousAddressBits = 48; internal const long kPreviousAddressMaskInWord = (1L << kPreviousAddressBits) - 1; - // Leftover bits (that were reclaimed from locking) - const int kLeftoverBitCount = 7; - // Other marker bits. Unused* means bits not yet assigned; use the highest number when assigning - const int kTombstoneBitOffset = kPreviousAddressBits + kLeftoverBitCount; + const int kKeyIsInlineBitOffset = kPreviousAddressBits; + const int kValueIsInlineBitOffset = kKeyIsInlineBitOffset + 1; + const int kValueIsObjectBitOffset = kValueIsInlineBitOffset + 1; + const int kUnused3BitOffset = kValueIsObjectBitOffset + 1; + const int kHasETagBitOffset = kUnused3BitOffset + 1; + const int kHasExpirationBitOffset = kHasETagBitOffset + 1; + const int kPreviousAddressIsOnDiskBitOffset = kHasExpirationBitOffset + 1; + + const int kTombstoneBitOffset = kPreviousAddressIsOnDiskBitOffset + 1; const int kValidBitOffset = kTombstoneBitOffset + 1; const int kSealedBitOffset = kValidBitOffset + 1; - const int kEtagBitOffset = kSealedBitOffset + 1; - const int kDirtyBitOffset = kEtagBitOffset + 1; + const int kUnused2BitOffset = kSealedBitOffset + 1; + const int kDirtyBitOffset = kUnused2BitOffset + 1; const int kFillerBitOffset = kDirtyBitOffset + 1; const int kInNewVersionBitOffset = kFillerBitOffset + 1; const int kModifiedBitOffset = kInNewVersionBitOffset + 1; const int kUnused1BitOffset = kModifiedBitOffset + 1; + const long kKeyIsInlineBitMask = 1L << kKeyIsInlineBitOffset; + const long kValueIsInlineBitMask = 1L << kValueIsInlineBitOffset; + const long kValueIsObjectBitMask = 1L << kValueIsObjectBitOffset; + const long kUnused3BitMask = 1L << kUnused3BitOffset; + const long kHasETagBitMask = 1L << kHasETagBitOffset; + const long kHasExpirationBitMask = 1L << kHasExpirationBitOffset; + const long kPreviousAddressIsOnDiskBitMask = 1L << kPreviousAddressIsOnDiskBitOffset; + const long kTombstoneBitMask = 1L << kTombstoneBitOffset; const long kValidBitMask = 1L << kValidBitOffset; const long kSealedBitMask = 1L << kSealedBitOffset; - const long kETagBitMask = 1L << kEtagBitOffset; + const long kUnused2BitMask = 1L << kUnused2BitOffset; const long kDirtyBitMask = 1L << kDirtyBitOffset; const long kFillerBitMask = 1L << kFillerBitOffset; const long kInNewVersionBitMask = 1L << kInNewVersionBitOffset; const long kModifiedBitMask = 1L << kModifiedBitOffset; const long kUnused1BitMask = 1L << kUnused1BitOffset; +#pragma warning restore IDE1006 // Naming Styles [FieldOffset(0)] private long word; @@ -62,7 +78,7 @@ public void WriteInfo(bool inNewVersion, long previousAddress) // Otherwise, Scan could return partial records (e.g. a checkpoint was taken that flushed midway through the record update). // - Revivification sets Sealed; we need to preserve it here. // We'll clear both on successful CAS. - InitializeToSealedAndInvalid(); + InitializeNewRecord(); PreviousAddress = previousAddress; if (inNewVersion) SetIsInNewVersion(); @@ -73,6 +89,8 @@ public void WriteInfo(bool inNewVersion, long previousAddress) public void ClearBitsForDiskImages() { // A Sealed record may become current again during recovery if the RCU-inserted record was not written to disk during a crash. So clear that bit here. + // Preserve Key/ValueIsInline as they are always inline for DiskLogRecord. Preserve ValueIsObject to indicate whether a value object should be deserialized + // or if the value should remain inline (and possibly overflow if copied to a LogRecord). word &= ~(kDirtyBitMask | kSealedBitMask); } @@ -80,7 +98,7 @@ public void ClearBitsForDiskImages() private static bool IsClosedWord(long word) => (word & (kValidBitMask | kSealedBitMask)) != kValidBitMask; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool IsClosedOrTombstoned(ref OperationStatus internalStatus) + internal readonly bool IsClosedOrTombstoned(ref OperationStatus internalStatus) { if ((word & (kValidBitMask | kSealedBitMask | kTombstoneBitMask)) != kValidBitMask) { @@ -111,7 +129,7 @@ public bool TrySeal(bool invalidate) // If this fails for any reason it means another record is trying to modify (perhaps revivify) it, so return false to RETRY_LATER. // If invalidate, we in a situation such as revivification freelisting where we want to make sure that removing Seal will not leave // it eligible to be Scanned after Recovery. - long expected_word = word; + var expected_word = word; if (IsClosedWord(expected_word)) return false; var new_word = expected_word | kSealedBitMask; @@ -127,9 +145,9 @@ public bool TrySeal(bool invalidate) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal bool TryResetModifiedAtomic() { - for (int spinCount = Constants.kMaxLockSpins; ; Thread.Yield()) + for (var spinCount = Constants.kMaxLockSpins; ; _ = Thread.Yield()) { - long expected_word = word; + var expected_word = word; if (IsClosedWord(expected_word)) return false; if ((expected_word & kModifiedBitMask) == 0) @@ -152,7 +170,7 @@ public bool TryUpdateAddress(long expectedPrevAddress, long newPrevAddress) return expected_word == Interlocked.CompareExchange(ref word, newRI.word, expected_word); } - public readonly bool IsNull() => word == 0; + public readonly bool IsNull => word == 0; public readonly bool Tombstone { @@ -180,18 +198,15 @@ public bool Valid public void ClearDirtyAtomic() { - for (; ; Thread.Yield()) + for (; ; _ = Thread.Yield()) { - long expected_word = word; // TODO: Interlocked.And is not supported in netstandard2.1 + var expected_word = word; // TODO: Interlocked.And is not supported in netstandard2.1 if (expected_word == Interlocked.CompareExchange(ref word, expected_word & ~kDirtyBitMask, expected_word)) break; } } - public readonly bool Dirty - { - get => (word & kDirtyBitMask) > 0; - } + public readonly bool Dirty => (word & kDirtyBitMask) > 0; public bool Modified { @@ -230,7 +245,12 @@ public readonly bool IsInNewVersion [MethodImpl(MethodImplOptions.AggressiveInlining)] public void SetInvalid() => word &= ~kValidBitMask; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void InitializeToSealedAndInvalid() => word = kSealedBitMask; // Does not include kValidBitMask + public void InitializeNewRecord() + { + // Initialize to Sealed and Invalid (do not include kValidBitMask) and to Inline Key and Value so no Oversize or ObjectId is expected. + word = kSealedBitMask | kKeyIsInlineBitMask | kValueIsInlineBitMask; // Does not include + } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void UnsealAndValidate() => word = (word & ~kSealedBitMask) | kValidBitMask; [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -241,9 +261,9 @@ public readonly bool IsInNewVersion [MethodImpl(MethodImplOptions.AggressiveInlining)] public void SetInvalidAtomic() { - for (; ; Thread.Yield()) + for (; ; _ = Thread.Yield()) { - long expected_word = word; // TODO: Interlocked.And is not supported in netstandard2.1 + var expected_word = word; // TODO: Interlocked.And is not supported in netstandard2.1 if (expected_word == Interlocked.CompareExchange(ref word, expected_word & ~kValidBitMask, expected_word)) return; } @@ -266,6 +286,42 @@ public long PreviousAddress set { word = (word & ~kPreviousAddressMaskInWord) | (value & kPreviousAddressMaskInWord); } } + public readonly bool HasETag => (word & kHasETagBitMask) != 0; + public void SetHasETag() => word |= kHasETagBitMask; + public void ClearHasETag() => word &= ~kHasETagBitMask; + + public readonly bool HasExpiration => (word & kHasExpirationBitMask) != 0; + public void SetHasExpiration() => word |= kHasExpirationBitMask; + public void ClearHasExpiration() => word &= ~kHasExpirationBitMask; + + // 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; + + // Note: ValueIsOveflow 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; + + 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 + public readonly bool ValueIsOverflow => !ValueIsInline && !ValueIsObject; + public void SetValueIsOverflow() => word &= ~(kValueIsInlineBitMask | kValueIsObjectBitMask); + + public readonly bool RecordIsInline => (word & (kKeyIsInlineBitMask | kValueIsInlineBitMask)) == (kKeyIsInlineBitMask | kValueIsInlineBitMask); + + internal bool PreviousAddressIsOnDisk + { + readonly get => (word & kPreviousAddressIsOnDiskBitMask) != 0; + set => word = value ? word | kPreviousAddressIsOnDiskBitMask : word & ~kPreviousAddressIsOnDiskBitMask; + } + public void SetPreviousAddressIsOnDisk() => word |= kPreviousAddressIsOnDiskBitMask; + public void ClearPreviousAddressIsOnDisk() => word &= ~kPreviousAddressIsOnDiskBitMask; + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static int GetLength() => kTotalSizeInBytes; @@ -275,21 +331,25 @@ internal bool Unused1 set => word = value ? word | kUnused1BitMask : word & ~kUnused1BitMask; } - public bool ETag + internal bool Unused2 { - readonly get => (word & kETagBitMask) != 0; - set => word = value ? word | kETagBitMask : word & ~kETagBitMask; + readonly get => (word & kUnused2BitMask) != 0; + set => word = value ? word | kUnused2BitMask : word & ~kUnused2BitMask; } - public void SetHasETag() => word |= kETagBitMask; - public void ClearHasETag() => word &= ~kETagBitMask; + internal bool Unused3 + { + readonly get => (word & kUnused3BitMask) != 0; + set => word = value ? word | kUnused3BitMask : word & ~kUnused3BitMask; + } public override readonly string ToString() { var paRC = IsReadCache(PreviousAddress) ? "(rc)" : string.Empty; static string bstr(bool value) => value ? "T" : "F"; return $"prev {AbsoluteAddress(PreviousAddress)}{paRC}, valid {bstr(Valid)}, tomb {bstr(Tombstone)}, seal {bstr(IsSealed)}," - + $" mod {bstr(Modified)}, dirty {bstr(Dirty)}, fill {bstr(HasFiller)}, etag {bstr(ETag)}, Un1 {bstr(Unused1)}"; + + $" 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)}, Un3 {bstr(Unused3)}"; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordMetadata.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordMetadata.cs index 397d78bf108..fd42aab6b05 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordMetadata.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/RecordMetadata.cs @@ -8,23 +8,17 @@ namespace Tsavorite.core /// public readonly struct RecordMetadata { - /// - /// The header of the record. - /// - public readonly RecordInfo RecordInfo; - /// /// The logical address of the record. /// public readonly long Address; - internal RecordMetadata(RecordInfo recordInfo, long address = Constants.kInvalidAddress) + internal RecordMetadata(long address = Constants.kInvalidAddress) { - RecordInfo = recordInfo; Address = address; } /// - public override string ToString() => $"ri {RecordInfo}, addr {Address}"; + public override string ToString() => $"addr {Address}"; } } \ 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 81b28f81f91..9e6c873f3ca 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/CallbackInfos.cs @@ -1,10 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System.Diagnostics; -using System.Runtime.CompilerServices; -using static Tsavorite.core.Utility; - namespace Tsavorite.core { /// @@ -24,7 +20,7 @@ public enum UpsertAction } /// - /// Information passed to record-update callbacks. + /// Information passed to record-update callbacks. /// public struct UpsertInfo { @@ -48,24 +44,6 @@ public struct UpsertInfo /// public int SessionID { get; internal set; } - /// - /// The header of the record. - /// - public RecordInfo RecordInfo { get; private set; } - - internal void SetRecordInfo(ref RecordInfo recordInfo) => RecordInfo = recordInfo; - - /// - /// The length of data in the value that is in use. Incoming, it is set by Tsavorite. - /// If an application wants to allow data to shrink and then grow again within the same record, it must set this to the correct length on output. - /// - public int UsedValueLength { get; set; } - - /// - /// The allocated length of the record value. - /// - public int FullValueLength { get; internal set; } - /// /// What actions Tsavorite should perform on a false return from the ISessionFunctions method /// @@ -82,65 +60,6 @@ public UpsertInfo(ref RMWInfo rmwInfo) KeyHash = rmwInfo.KeyHash; Action = UpsertAction.Default; } - - /// - /// Retrieve the extra value length from the record, if present, and then clear it to ensure consistent log scan during in-place update. - /// - /// Reference to the record value - /// The currently-used length of the record value - /// The record header - /// The type of the value - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly unsafe void ClearExtraValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength) - { - Debug.Assert(usedValueLength == UsedValueLength, $"UpsertInfo: usedValueLength ({usedValueLength}) != this.UsedValueLength ({UsedValueLength})"); - StaticClearExtraValueLength(ref recordInfo, ref recordValue, usedValueLength); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static unsafe void StaticClearExtraValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength) - { - if (!recordInfo.HasFiller) - return; - - var valueAddress = (long)Unsafe.AsPointer(ref recordValue); - int* extraLengthPtr = (int*)(valueAddress + RoundUp(usedValueLength, sizeof(int))); - - *extraLengthPtr = 0; - recordInfo.ClearHasFiller(); - } - - /// - /// Set the extra value length, if any, into the record past the used value length. - /// - /// The record header - /// Reference to the record value - /// The currently-used length of the record value - /// The type of the value - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public unsafe void SetUsedValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength) - { - StaticSetUsedValueLength(ref recordInfo, ref recordValue, usedValueLength, FullValueLength); - UsedValueLength = usedValueLength; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static unsafe void StaticSetUsedValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength, int fullValueLength) - { - // Note: This is only called for variable-length types, and for those we have ensured the location of recordValue is pinned. - long valueAddress = (long)Unsafe.AsPointer(ref recordValue); - Debug.Assert(!recordInfo.HasFiller, "Filler should have been cleared by ClearExtraValueLength()"); - - usedValueLength = RoundUp(usedValueLength, sizeof(int)); - int extraValueLength = fullValueLength - usedValueLength; - if (extraValueLength >= sizeof(int)) - { - int* extraValueLengthPtr = (int*)(valueAddress + usedValueLength); - Debug.Assert(*extraValueLengthPtr == 0 || *extraValueLengthPtr == extraValueLength, "existing ExtraValueLength should be 0 or the same value"); - *extraValueLengthPtr = extraValueLength; - recordInfo.SetHasFiller(); - } - } } /// @@ -170,7 +89,7 @@ public enum RMWAction } /// - /// Information passed to record-update callbacks. + /// Information passed to record-update callbacks. /// public struct RMWInfo { @@ -195,69 +114,20 @@ public struct RMWInfo /// public int SessionID { get; internal set; } - /// - /// The header of the record. - /// - public RecordInfo RecordInfo { get; private set; } - - internal void SetRecordInfo(ref RecordInfo recordInfo) => RecordInfo = recordInfo; - internal void ClearRecordInfo() => RecordInfo = default; - - /// - /// The length of data in the value that is in use. Incoming, it is set by Tsavorite. - /// If an application wants to allow data to shrink and then grow again within the same record, it must set this to the correct length on output. - /// - public int UsedValueLength { get; set; } - - /// - /// The allocated length of the record value. - /// - public int FullValueLength { get; internal set; } - - public int FullRecordLength { get; internal set; } - /// /// If set true by CopyUpdater, the source record for the RCU will not be elided from the tag chain even if this is otherwise possible. /// public bool PreserveCopyUpdaterSourceRecord { get; set; } /// - /// Whether the call is from sync or async (pending) path + /// If set true by CopyUpdater, the source ValueObject will be cleared immediately (to manage object size tracking most effectively). /// - public bool IsFromPending { get; internal set; } + public bool ClearSourceValueObject { get; set; } /// /// What actions Tsavorite should perform on a false return from the ISessionFunctions method /// public RMWAction Action { get; set; } - - /// - /// Retrieve the extra value length from the record, if present, and then clear it to ensure consistent log scan during in-place update. - /// - /// Reference to the record header - /// Reference to the record value - /// The currently-used length of the record value - /// The type of the value - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ClearExtraValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength) - { - Debug.Assert(usedValueLength == UsedValueLength, $"RMWInfo: usedValueLength ({usedValueLength}) != this.UsedValueLength ({UsedValueLength})"); - UpsertInfo.StaticClearExtraValueLength(ref recordInfo, ref recordValue, usedValueLength); - } - - /// - /// Set the extra value length, if any, into the record past the used value length. - /// - /// Reference to the record header - /// Reference to the record value - /// The currently-used length of the record value - /// The type of the value - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public unsafe void SetUsedValueLength(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength) - { - UpsertInfo.StaticSetUsedValueLength(ref recordInfo, ref recordValue, usedValueLength, FullValueLength); - UsedValueLength = usedValueLength; - } } /// @@ -276,7 +146,7 @@ public enum DeleteAction CancelOperation } /// - /// Information passed to record-update callbacks. + /// Information passed to record-update callbacks. /// public struct DeleteInfo { @@ -300,24 +170,6 @@ public struct DeleteInfo /// public int SessionID { get; internal set; } - /// - /// The header of the record. - /// - public RecordInfo RecordInfo { get; private set; } - - internal void SetRecordInfo(ref RecordInfo recordInfo) => RecordInfo = recordInfo; - - /// - /// The length of data in the value that is in use. Incoming, it is set by Tsavorite to the result. - /// If an application wants to allow data to shrink and then grow again within the same record, it must set this to the correct length on output. - /// - public int UsedValueLength { get; set; } - - /// - /// The allocated length of the record value. - /// - public int FullValueLength { get; internal set; } - /// /// What actions Tsavorite should perform on a false return from the ISessionFunctions method /// @@ -346,7 +198,7 @@ public enum ReadAction } /// - /// Information passed to record-read callbacks. + /// Information passed to record-read callbacks. /// public struct ReadInfo { @@ -360,13 +212,6 @@ public struct ReadInfo /// public long Address { get; internal set; } - /// - /// The header of the record. - /// - public RecordInfo RecordInfo { get; private set; } - - internal void SetRecordInfo(ref RecordInfo recordInfo) => RecordInfo = recordInfo; - /// /// Whether the call is from sync or async (pending) path /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/DataContractObjectSerializer.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/DataContractObjectSerializer.cs deleted file mode 100644 index 2dddae94f34..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/DataContractObjectSerializer.cs +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.IO; -using System.Runtime.Serialization; -using System.Xml; - -namespace Tsavorite.core -{ - /// - /// Serializer (for class types) based on DataContract - /// - /// - public sealed class DataContractObjectSerializer : BinaryObjectSerializer - { - private static readonly DataContractSerializer serializer = new DataContractSerializer(typeof(T)); - - /// - /// Deserialize - /// - /// - public override void Deserialize(out T obj) - { - int count = reader.ReadInt32(); - var byteArray = reader.ReadBytes(count); - using var ms = new MemoryStream(byteArray); - using var _reader = XmlDictionaryReader.CreateBinaryReader(ms, XmlDictionaryReaderQuotas.Max); - obj = (T)serializer.ReadObject(_reader); - } - - /// - /// Serialize - /// - /// - public override void Serialize(ref T obj) - { - using var ms = new MemoryStream(); - using (var _writer = XmlDictionaryWriter.CreateBinaryWriter(ms, null, null, false)) - serializer.WriteObject(_writer, obj); - writer.Write((int)ms.Position); - writer.Write(ms.ToArray()); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IKeyComparer.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IKeyComparer.cs index 1d5299b0b24..a3ba3eb3a20 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IKeyComparer.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IKeyComparer.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Collections.Generic; namespace Tsavorite.core @@ -8,20 +9,19 @@ namespace Tsavorite.core /// /// Defines methods to support the comparison of Tsavorite keys for equality. /// - /// The type of keys to compare. - /// This comparer differs from the built-in in that it implements a 64-bit hash code - public interface IKeyComparer + /// This comparer differs from the built-in in that it implements a 64-bit hash code + public interface IKeyComparer { /// /// Get 64-bit hash code /// - long GetHashCode64(ref T key); + long GetHashCode64(ReadOnlySpan key); /// /// Equality comparison /// /// Left side /// Right side - bool Equals(ref T k1, ref T k2); + bool Equals(ReadOnlySpan k1, ReadOnlySpan k2); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IObjectSerializer.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IObjectSerializer.cs index 42770386641..8aaf61ae1a0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IObjectSerializer.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/IObjectSerializer.cs @@ -22,7 +22,7 @@ public interface IObjectSerializer /// Serialize object /// /// - void Serialize(ref T obj); + void Serialize(T obj); /// /// End serialization to given stream @@ -69,7 +69,7 @@ public abstract class BinaryObjectSerializer : IObjectSerializer public void BeginSerialize(Stream stream) => writer = new BinaryWriter(stream, new UTF8Encoding(), true); /// Serialize - public abstract void Serialize(ref T obj); + public abstract void Serialize(T obj); /// End serialize public void EndSerialize() => writer.Dispose(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctions.cs index f9c3990984c..fcff3f7f4e3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctions.cs @@ -1,94 +1,150 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// /// Callback functions to Tsavorite /// - /// - /// - /// - /// - /// - public interface ISessionFunctions + public interface ISessionFunctions : IVariableLengthInput { #region Reads /// - /// Non-concurrent reader. + /// Read the record by copying all or part of it to . /// - /// The key for the record to be read - /// The user input for computing from - /// The value for the record being read - /// The location where is to be copied + /// The log record being read + /// The user input for computing from the record value + /// Receives the output of the operation, if any /// Information about this read operation and its context /// True if the value was available, else false (e.g. the value was expired) - bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo); - - /// - /// Concurrent reader - /// - /// The key for the record to be read - /// The user input for computing from - /// The value for the record being read - /// The location where is to be copied - /// Information about this read operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification - /// True if the value was available, else false (e.g. the value was expired) - bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo); + bool Reader(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord; /// /// Read completion /// - /// The key for this record + /// The log record that was read from disk /// The user input that was used in the read operation /// The result of the read operation; if this is a struct, then it will be a temporary and should be copied to /// The application context passed through the pending operation /// The result of the pending operation - /// Metadata for the record; may be used to obtain .PreviousAddress when doing iterative reads - void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); + /// Metadata for the record + void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); #endregion reads #region Upserts /// - /// Non-concurrent writer; called on an Upsert that does not find the key so does an insert or finds the key's record in the immutable region so does a read/copy/update (RCU). + /// Non-concurrent writer for Span value; called on an Upsert that does not find the key so does an insert or finds the key's record in the immutable region so does a read/copy/update (RCU). + /// + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing + /// The input Span to be copied to the record value + /// The location where the result of the update may be placed + /// Information about this update operation and its context + /// True if the write was performed, else false (e.g. cancellation) + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + + /// + /// Non-concurrent writer for Object values; called on an Upsert that does not find the key so does an insert or finds the key's record in the immutable region so does a read/copy/update (RCU). + /// + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing + /// The input Object to be copied to the record value + /// The location where the result of the update may be placed + /// Information about this update operation and its context + /// True if the write was performed, else false (e.g. cancellation) + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + + /// + /// Non-concurrent writer for Object values; called on an Upsert that does not find the key so does an insert or finds the key's record in the immutable region so does a read/copy/update (RCU). /// - /// The key for this record - /// The user input to be used for computing - /// The previous value to be copied/updated - /// The destination to be updated; because this is an copy to a new location, there is no previous value there. + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing + /// The log record passed to Upsert, to be copied to the destination record /// The location where the result of the update may be placed /// Information about this update operation and its context - /// The operation for which this write is being done - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// True if the write was performed, else false (e.g. cancellation) - bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo); + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; + + /// + /// Called after InitialWriter when a record has been successfully inserted at the tail of the log. + /// + /// The destination log record + /// The size information for this record's fields + /// The user input that was used to compute + /// The input Span that was to be copied to the record value + /// The location where the result of the update may be placed + /// Information about this update operation and its context + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + + /// + /// Called after InitialWriter when a record has been successfully inserted at the tail of the log. + /// + /// The destination log record + /// The size information for this record's fields + /// The user input that was used to compute + /// The input Object that was to be copied to the record value + /// The location where the result of the update may be placed + /// Information about this update operation and its context + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + + /// + /// Called after InitialWriter when a record has been successfully inserted at the tail of the log. + /// + /// The destination log record + /// The size information for this record's fields + /// The user input that was used to compute + /// The input LogRecord that was to be copied to the record value + /// The location where the result of the update may be placed + /// Information about this update operation and its context + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; + + /// + /// Concurrent writer; called on an Upsert that is in-place updating a record in the mutable range. + /// + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing the destination record's value + /// The Span value passed to Upsert, to be copied to the destination record + /// The location where the result of the update may be placed + /// Information about this update operation and its context + /// True if the value was written, else false + /// If the value is shrunk in-place, the caller must first zero the data that is no longer used, to ensure log-scan correctness. + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan newValue, ref TOutput output, ref UpsertInfo upsertInfo); /// - /// Called after SingleWriter when a record containing an upsert of a new key has been successfully inserted at the tail of the log. + /// Concurrent writer; called on an Upsert that is in-place updating a record in the mutable range. /// - /// The key for this record - /// The user input that was used to compute - /// The previous value to be copied/updated - /// The destination to be updated; because this is an copy to a new location, there is no previous value there. + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing the destination record's value + /// The value passed to Upsert, to be copied to the destination record /// The location where the result of the update may be placed /// Information about this update operation and its context - /// The operation for which this write is being done - void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason); + /// True if the value was written, else false + /// If the value is shrunk in-place, the caller must first zero the data that is no longer used, to ensure log-scan correctness. + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject newValue, ref TOutput output, ref UpsertInfo upsertInfo); /// - /// Concurrent writer; called on an Upsert that finds the record in the mutable range. + /// Concurrent writer; called on an Upsert that is in-place updating a record in the mutable range. The caller should be aware of ETag and Expiration in the source record. /// - /// The key for the record to be written - /// The user input to be used for computing - /// The value to be copied to - /// The location where is to be copied; because this method is called only for in-place updates, there is a previous value there. + /// The destination log record + /// The size information for this record's fields + /// The user input to be used for computing the destination record's value + /// The log record passed to Upsert, to be copied to the destination record /// The location where the result of the update may be placed /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// True if the value was written, else false /// If the value is shrunk in-place, the caller must first zero the data that is no longer used, to ensure log-scan correctness. - bool ConcurrentWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo); + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; #endregion Upserts #region RMWs @@ -96,149 +152,127 @@ public interface ISessionFunctions /// /// Whether we need to invoke initial-update for RMW /// - /// The key for this record + /// The key for this record; this is the key passed to Upsert as we don't have a log record yet. /// The user input to be used for computing the updated value /// The location where the result of the operation is to be copied /// Information about this update operation and its context - bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); + bool NeedInitialUpdate(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); /// /// Initial update for RMW (insert at the tail of the log). /// - /// The key for this record - /// The user input to be used for computing the updated - /// The destination to be updated; because this is an insert, there is no previous value there. - /// The location where the result of the operation on is to be copied + /// The destination log record + /// The size information for this record's fields + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// True if the write was performed, else false (e.g. cancellation) - bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); + bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); /// /// Called after a record containing an initial update for RMW has been successfully inserted at the tail of the log. /// - /// The key for this record - /// The user input to be used for computing the updated - /// The destination to be updated; because this is an insert, there is no previous value there. - /// The location where the result of the operation on is to be copied + /// The log record that was created + /// The size information for this record's fields + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context - void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo); + void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); #endregion InitialUpdater #region CopyUpdater /// /// Whether we need to invoke copy-update for RMW /// - /// The key for this record + /// The source record being copied from /// The user input to be used for computing the updated value - /// The existing value that would be copied. - /// The location where the result of the operation on is to be copied + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context - bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo); + bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; /// /// Copy-update for RMW (RCU (Read-Copy-Update) to the tail of the log) /// - /// The key for this record - /// The user input to be used for computing from - /// The previous value to be copied/updated - /// The destination to be updated; because this is an copy to a new location, there is no previous value there. - /// The location where is to be copied + /// The source record being copied from + /// The destination log record being created + /// The size information for this record's fields + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// True if the write was performed, else false (e.g. cancellation) - bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); + bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; /// /// Called after a record containing an RCU (Read-Copy-Update) for RMW has been successfully inserted at the tail of the log. /// - /// The key for this record - /// The user input to be used for computing from - /// The previous value to be copied/updated; may also be disposed here if appropriate - /// The destination to be updated; because this is an copy to a new location, there is no previous value there. - /// The location where is to be copied + /// The source record being copied from + /// The destination log record being created + /// The size information for this record's fields + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context /// This is the only Post* method that returns non-void. The bool functions the same as CopyUpdater; this is because we do not want to modify /// objects in-memory until we know the "insert at tail" is successful. Therefore, we allow a false return as a signal to inspect /// and handle . - bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo); + bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; #endregion CopyUpdater #region InPlaceUpdater /// /// In-place update for RMW /// - /// The key for this record - /// The user input to be used for computing the updated - /// The destination to be updated; because this is an in-place update, there is a previous value there. - /// The location where the result of the operation on is to be copied + /// The log record that is being updated + /// The size information for this record's fields + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// True if the value was successfully updated, else false (e.g. the value was expired) /// If the value is shrunk in-place, the caller must first zero the data that is no longer used, to ensure log-scan correctness. - bool InPlaceUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); + bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); #endregion InPlaceUpdater - #region Variable-length value size - /// - /// Length of resulting value object when performing RMW modification of value using given input - /// - int GetRMWModifiedValueLength(ref TValue value, ref TInput input); - - /// - /// Initial expected length of value object when populated by RMW using given input - /// - int GetRMWInitialValueLength(ref TInput input); - - /// - /// Length of resulting value object when performing Upsert of value using given input - /// - int GetUpsertValueLength(ref TValue value, ref TInput input); - #endregion Variable-length value size - /// /// RMW completion /// - /// The key for this record - /// The user input that was used to perform the modification - /// The result of the RMW operation; if this is a struct, then it will be a temporary and should be copied to + /// The log record that was read from disk + /// The user input to be used to create the destination record's value + /// The location where the output of the operation, if any, is to be copied /// The application context passed through the pending operation /// The result of the pending operation /// The metadata of the modified or inserted record - void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); + void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); #endregion RMWs #region Deletes /// /// Single deleter; called on a Delete that does not find the record in the mutable range and so inserts a new record. /// - /// The key for the record to be deleted - /// The value for the record being deleted; because this method is called only for in-place updates, there is a previous value there. Usually this is ignored or assigned 'default'. + /// The log record that is being created with a tombstone /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification /// For Object Value types, Dispose() can be called here. If recordInfo.Invalid is true, this is called after the record was allocated and populated, but could not be appended at the end of the log. /// True if the deleted record should be added, else false (e.g. cancellation) - bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo); + bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); /// /// Called after a record marking a Delete (with Tombstone set) has been successfully inserted at the tail of the log. /// - /// The key for the record that was deleted + /// The log record that was created with a tombstone /// Information about this update operation and its context /// This does not have the address of the record that contains the value at 'key'; Delete does not retrieve records below HeadAddress, so /// the last record we have in the 'key' chain may belong to 'key' or may be a collision. - void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo); + void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); /// /// Concurrent deleter; called on a Delete that finds the record in the mutable range. /// - /// The key for the record to be deleted - /// The value for the record being deleted; because this method is called only for in-place updates, there is a previous value there. Usually this is ignored or assigned 'default'. + /// The log record that is being deleted in-place /// Information about this update operation and its context - /// A reference to the RecordInfo for the record; used for variable-length record length modification - /// For Object Value types, Dispose() can be called here. If recordInfo.Invalid is true, this is called after the record was allocated and populated, but could not be appended at the end of the log. + /// For Object Value types, Dispose() can be called here. If logRecord.Info.Invalid is true, this is called after the record was allocated and populated, but could not be appended at the end of the log. /// True if the value was successfully deleted, else false (e.g. the record was sealed) - bool ConcurrentDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo); + bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); #endregion Deletes #region Utilities @@ -254,19 +288,14 @@ public interface ISessionFunctions /// /// Callback functions to Tsavorite (two-param version) /// - /// - /// - public interface ISessionFunctions : ISessionFunctions + public interface ISessionFunctions : ISessionFunctions { } /// /// Callback functions to Tsavorite (two-param version with context) /// - /// - /// - /// - public interface ISessionFunctions : ISessionFunctions + public interface ISessionFunctions : ISessionFunctions { } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs index 2f7dc84ea00..4f40f9b66fd 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs @@ -1,55 +1,69 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// /// Provides thread management and all callbacks. A wrapper for IFunctions and additional methods called by TsavoriteImpl; the wrapped /// IFunctions methods provide additional parameters to support the wrapper functionality, then call through to the user implementations. /// - internal interface ISessionFunctionsWrapper : IVariableLengthInput - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal interface ISessionFunctionsWrapper : IVariableLengthInput + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - bool IsManualLocking { get; } - TsavoriteKV Store { get; } + bool IsTransactionalLocking { get; } + TsavoriteKV Store { get; } #region Reads - bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo); - bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo); - void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); + bool Reader(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput dst, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord; + void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); #endregion reads #region Upserts - bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo); - void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo); - bool ConcurrentWriter(long physicalAddress, ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo); + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo); + bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord; #endregion Upserts #region RMWs #region InitialUpdater - bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); - bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); - void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rMWInfo, ref RecordInfo recordInfo); + bool NeedInitialUpdate(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); + bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo); + void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rMWInfo); #endregion InitialUpdater #region CopyUpdater - bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo); - bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); - bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo); + bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; + bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; + bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord; #endregion CopyUpdater #region InPlaceUpdater - bool InPlaceUpdater(long physicalAddress, ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, out OperationStatus status, ref RecordInfo recordInfo); + bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo, out OperationStatus status); #endregion InPlaceUpdater - void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); + void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata); #endregion RMWs #region Deletes - bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo); - void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo); - bool ConcurrentDeleter(long physicalAddress, ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo, out int fullRecordLength); + bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); + void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); + bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo); #endregion Deletes #region Utilities @@ -57,22 +71,20 @@ internal interface ISessionFunctionsWrapper stackCtx); - bool TryLockTransientShared(ref TKey key, ref OperationStackContext stackCtx); - void UnlockTransientExclusive(ref TKey key, ref OperationStackContext stackCtx); - void UnlockTransientShared(ref TKey key, ref OperationStackContext stackCtx); - #endregion + #region Ephemeral locking + bool TryLockEphemeralExclusive(ref OperationStackContext stackCtx); + bool TryLockEphemeralShared(ref OperationStackContext stackCtx); + void UnlockEphemeralExclusive(ref OperationStackContext stackCtx); + void UnlockEphemeralShared(ref OperationStackContext stackCtx); + #endregion #region Epoch control void UnsafeResumeThread(); void UnsafeSuspendThread(); #endregion - bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false); - - TsavoriteKV.TsavoriteExecutionContext Ctx { get; } + bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false); - IHeapContainer GetHeapContainer(ref TInput input); + TsavoriteKV.TsavoriteExecutionContext Ctx { get; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionLocker.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionLocker.cs index 8c83129da63..120dbe85089 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionLocker.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionLocker.cs @@ -10,106 +10,106 @@ namespace Tsavorite.core /// Provides thread management and all callbacks. A wrapper for ISessionFunctions and additional methods called by TsavoriteImpl; the wrapped /// ISessionFunctions methods provide additional parameters to support the wrapper functionality, then call through to the user implementations. /// - public interface ISessionLocker - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public interface ISessionLocker + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - bool IsManualLocking { get; } + bool IsTransactionalLocking { get; } - bool TryLockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx); - bool TryLockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx); - void UnlockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx); - void UnlockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx); + bool TryLockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx); + bool TryLockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx); + void UnlockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx); + void UnlockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx); } /// - /// Basic (non-lockable) sessions must do transient locking. + /// Basic (non-transactional) sessions must do Ephemeral locking. /// /// /// This struct contains no data fields; SessionFunctionsWrapper redirects with its ClientSession. /// - internal struct BasicSessionLocker : ISessionLocker - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal struct BasicSessionLocker : ISessionLocker + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public bool IsManualLocking => false; + public bool IsTransactionalLocking => false; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) + public bool TryLockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) { if (!store.LockTable.TryLockExclusive(ref stackCtx.hei)) return false; - stackCtx.recSrc.SetHasTransientXLock(); + stackCtx.recSrc.SetHasEphemeralXLock(); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx) + public bool TryLockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx) { if (!store.LockTable.TryLockShared(ref stackCtx.hei)) return false; - stackCtx.recSrc.SetHasTransientSLock(); + stackCtx.recSrc.SetHasEphemeralSLock(); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) + public void UnlockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) { store.LockTable.UnlockExclusive(ref stackCtx.hei); - stackCtx.recSrc.ClearHasTransientXLock(); + stackCtx.recSrc.ClearHasEphemeralXLock(); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx) + public void UnlockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx) { store.LockTable.UnlockShared(ref stackCtx.hei); - stackCtx.recSrc.ClearHasTransientSLock(); + stackCtx.recSrc.ClearHasEphemeralSLock(); } } /// - /// Lockable sessions are manual locking and thus must have already locked the record prior to an operation on it, so assert that. + /// Transactional sessions must have already locked the record prior to an operation on it, so assert that. /// - internal struct LockableSessionLocker : ISessionLocker - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal struct TransactionalSessionLocker : ISessionLocker + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - public bool IsManualLocking => true; + public bool IsTransactionalLocking => true; [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) + public bool TryLockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) { Debug.Assert(store.LockTable.IsLockedExclusive(ref stackCtx.hei), - $"Attempting to use a non-XLocked key in a Lockable context (requesting XLock):" + $"Attempting to use a non-XLocked key in a Transactional context (requesting XLock):" + $" XLocked {store.LockTable.IsLockedExclusive(ref stackCtx.hei)}," + $" Slocked {store.LockTable.IsLockedShared(ref stackCtx.hei)}"); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryLockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx) + public bool TryLockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx) { Debug.Assert(store.LockTable.IsLocked(ref stackCtx.hei), - $"Attempting to use a non-Locked (S or X) key in a Lockable context (requesting SLock):" + $"Attempting to use a non-Locked (S or X) key in a Transactional context (requesting SLock):" + $" XLocked {store.LockTable.IsLockedExclusive(ref stackCtx.hei)}," + $" Slocked {store.LockTable.IsLockedShared(ref stackCtx.hei)}"); return true; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) + public void UnlockEphemeralExclusive(TsavoriteKV store, ref OperationStackContext stackCtx) { Debug.Assert(store.LockTable.IsLockedExclusive(ref stackCtx.hei), - $"Attempting to unlock a non-XLocked key in a Lockable context (requesting XLock):" + $"Attempting to unlock a non-XLocked key in a Transactional context (requesting XLock):" + $" XLocked {store.LockTable.IsLockedExclusive(ref stackCtx.hei)}," + $" Slocked {store.LockTable.IsLockedShared(ref stackCtx.hei)}"); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnlockTransientShared(TsavoriteKV store, ref OperationStackContext stackCtx) + public void UnlockEphemeralShared(TsavoriteKV store, ref OperationStackContext stackCtx) { Debug.Assert(store.LockTable.IsLockedShared(ref stackCtx.hei), - $"Attempting to use a non-XLocked key in a Lockable context (requesting XLock):" + $"Attempting to use a non-XLocked key in a Transactional context (requesting XLock):" + $" XLocked {store.LockTable.IsLockedExclusive(ref stackCtx.hei)}," + $" Slocked {store.LockTable.IsLockedShared(ref stackCtx.hei)}"); } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/KeyComparers.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/KeyComparers.cs deleted file mode 100644 index 206aaaf7baa..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/KeyComparers.cs +++ /dev/null @@ -1,195 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.Diagnostics; - -namespace Tsavorite.core -{ - internal static class KeyComparers - { - public static IKeyComparer Get() - { - if (typeof(T) == typeof(string)) - return (IKeyComparer)(object)StringKeyComparer.Instance; - else if (typeof(T) == typeof(byte[])) - return (IKeyComparer)(object)ByteArrayKeyComparer.Instance; - else if (typeof(T) == typeof(long)) - return (IKeyComparer)(object)LongKeyComparer.Instance; - else if (typeof(T) == typeof(int)) - return (IKeyComparer)(object)IntKeyComparer.Instance; - else if (typeof(T) == typeof(Guid)) - return (IKeyComparer)(object)GuidKeyComparer.Instance; - else if (typeof(T) == typeof(SpanByte)) - return (IKeyComparer)(object)SpanByteComparer.Instance; - else - { - Debug.WriteLine("***WARNING*** Creating default Tsavorite key equality comparer based on potentially slow EqualityComparer.Default." - + "To avoid this, provide a comparer (ITsavoriteEqualityComparer) as an argument to Tsavorite's constructor, or make Key implement the interface ITsavoriteEqualityComparer"); - return DefaultKeyComparer.Instance; - } - } - } - - /// - /// Deterministic equality comparer for strings - /// - public sealed class StringKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly StringKeyComparer Instance = new(); - - /// - public bool Equals(ref string key1, ref string key2) - { - // Use locals in case the record space is cleared. - string k1 = key1, k2 = key2; - return (k1 is null || k2 is null) ? false : k1 == k2; - } - - /// - public unsafe long GetHashCode64(ref string key) - { - // Use locals in case the record space is cleared. - string k = key; - if (k is null) - return 0; - - fixed (char* c = k) - { - return Utility.HashBytes((byte*)c, key.Length * sizeof(char)); - } - } - } - - /// - /// Deterministic equality comparer for longs - /// - public sealed class LongKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly LongKeyComparer Instance = new(); - - /// - public bool Equals(ref long k1, ref long k2) => k1 == k2; - - /// - public long GetHashCode64(ref long k) => Utility.GetHashCode(k); - } - - /// - /// Deterministic equality comparer for longs - /// - public sealed class IntKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly IntKeyComparer Instance = new(); - - /// - public bool Equals(ref int k1, ref int k2) => k1 == k2; - - /// - public long GetHashCode64(ref int k) => Utility.GetHashCode(k); - } - - /// - /// Deterministic equality comparer for longs - /// - public sealed class GuidKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly GuidKeyComparer Instance = new(); - - /// - public bool Equals(ref Guid k1, ref Guid k2) => k1 == k2; - - /// - public unsafe long GetHashCode64(ref Guid k) - { - var _k = k; - var pGuid = (long*)&_k; - return pGuid[0] ^ pGuid[1]; - } - } - - /// - /// Deterministic equality comparer for byte[] - /// - public sealed class ByteArrayKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly ByteArrayKeyComparer Instance = new(); - - /// - public bool Equals(ref byte[] key1, ref byte[] key2) => key1.AsSpan().SequenceEqual(key2); - - /// - public unsafe long GetHashCode64(ref byte[] key) - { - // Use locals in case the record space is cleared. - byte[] k = key; - if (k is null) - return 0; - - fixed (byte* b = k) - { - return Utility.HashBytes(b, k.Length); - } - } - } - - /// - /// No-op equality comparer for Empty (used by TsavoriteLog) - /// - public sealed class EmptyKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly EmptyKeyComparer Instance = new(); - - /// - public bool Equals(ref Empty key1, ref Empty key2) => throw new NotImplementedException(); - - /// - public long GetHashCode64(ref Empty key) => throw new NotImplementedException(); - } - - /// - /// Low-performance Tsavorite equality comparer wrapper around EqualityComparer.Default - /// - /// - internal sealed class DefaultKeyComparer : IKeyComparer - { - /// - /// The default instance. - /// - /// Used to avoid allocating new comparers. - public static readonly DefaultKeyComparer Instance = new(); - - private static readonly EqualityComparer DefaultEC = EqualityComparer.Default; - - /// - public bool Equals(ref T k1, ref T k2) => DefaultEC.Equals(k1, k2); - - /// - public long GetHashCode64(ref T k) => Utility.GetHashCode(DefaultEC.GetHashCode(k)); - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ObjectSerializer.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ObjectSerializer.cs deleted file mode 100644 index 7802a82bf78..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ObjectSerializer.cs +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; - -namespace Tsavorite.core -{ - internal static class ObjectSerializer - { - public static Func> Get() - { - if (typeof(T) == typeof(string)) - return () => (IObjectSerializer)new StringBinaryObjectSerializer(); - else if (typeof(T) == typeof(byte[])) - return () => (IObjectSerializer)new ByteArrayBinaryObjectSerializer(); - else - return () => new DataContractObjectSerializer(); - } - } - - internal class StringBinaryObjectSerializer : BinaryObjectSerializer - { - public override void Deserialize(out string obj) => obj = reader.ReadString(); - public override void Serialize(ref string obj) => writer.Write(obj); - } - - internal class ByteArrayBinaryObjectSerializer : BinaryObjectSerializer - { - public override void Deserialize(out byte[] obj) => obj = reader.ReadBytes(reader.ReadInt32()); - public override void Serialize(ref byte[] obj) - { - writer.Write(obj.Length); - writer.Write(obj); - } - } -} \ No newline at end of file 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 2d222e4b33d..d91ea2baea7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/SessionFunctionsBase.cs @@ -10,113 +10,120 @@ namespace Tsavorite.core /// /// Default empty functions base class to make it easy for users to provide their own implementation of ISessionFunctions /// - /// - /// - /// - /// - /// - public abstract class SessionFunctionsBase : ISessionFunctions + public abstract class SessionFunctionsBase : ISessionFunctions { /// - public virtual bool ConcurrentReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) => true; - /// - public virtual bool SingleReader(ref TKey key, ref TInput input, ref TValue value, ref TOutput dst, ref ReadInfo readInfo) => true; + public virtual bool Reader(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref ReadInfo readInfo) + where TSourceLogRecord : ISourceLogRecord + => true; /// - public virtual bool ConcurrentWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { dst = src; return true; } - /// - public virtual bool SingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) { dst = src; return true; } - /// - public virtual void PostSingleWriter(ref TKey key, ref TInput input, ref TValue src, ref TValue dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason) { } + public virtual bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration, which will come from TInput in fuller implementations. + return logRecord.TrySetValueSpan(srcValue, ref sizeInfo); + } /// - public virtual bool InitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; + public virtual bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration, which will come from TInput in fuller implementations. + return logRecord.TrySetValueObject(srcValue, ref sizeInfo); + } + + public virtual bool InPlaceWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + // This includes ETag and Expiration + return dstLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + } + /// - public virtual void PostInitialUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo) { } + public virtual bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration, which will come from TInput in fuller implementations. + return dstLogRecord.TrySetValueSpan(srcValue, ref sizeInfo); + } + /// - public virtual bool NeedInitialUpdate(ref TKey key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; + public virtual bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) + { + // This does not try to set ETag or Expiration, which will come from TInput in fuller implementations. + return dstLogRecord.TrySetValueObject(srcValue, ref sizeInfo); + } + /// - public virtual bool NeedCopyUpdate(ref TKey key, ref TInput input, ref TValue oldValue, ref TOutput output, ref RMWInfo rmwInfo) => true; + public virtual bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { + // This includes ETag and Expiration + return dstLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); + } + /// - public virtual bool CopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; + public virtual void PostInitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ReadOnlySpan srcValue, ref TOutput output, ref UpsertInfo upsertInfo) { } /// - public virtual bool PostCopyUpdater(ref TKey key, ref TInput input, ref TValue oldValue, ref TValue newValue, ref TOutput output, ref RMWInfo rmwInfo) => true; + public virtual void PostInitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, IHeapObject srcValue, ref TOutput output, ref UpsertInfo upsertInfo) { } /// - public virtual bool InPlaceUpdater(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; + public virtual void PostInitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo) + where TSourceLogRecord : ISourceLogRecord + { } /// - public virtual bool SingleDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) { value = default; return true; } - public virtual void PostSingleDeleter(ref TKey key, ref DeleteInfo deleteInfo) { } - public virtual bool ConcurrentDeleter(ref TKey key, ref TValue value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - - public virtual void ReadCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } + public virtual bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; + /// + public virtual void PostInitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) { } /// - public virtual void RMWCompletionCallback(ref TKey key, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } + public virtual bool NeedInitialUpdate(ReadOnlySpan key, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; /// - public virtual int GetRMWModifiedValueLength(ref TValue value, ref TInput input) => throw new TsavoriteException("GetRMWModifiedValueLength is only available for SpanByte Functions"); + public virtual bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; /// - public virtual int GetRMWInitialValueLength(ref TInput input) => throw new TsavoriteException("GetRMWInitialValueLength is only available for SpanByte Functions"); + public virtual bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; /// - public virtual int GetUpsertValueLength(ref TValue value, ref TInput input) => throw new TsavoriteException("GetUpsertValueLength is only available for SpanByte Functions"); - + public virtual bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) + where TSourceLogRecord : ISourceLogRecord + => true; /// - public virtual void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } - } - - /// - /// Default empty functions base class to make it easy for users to provide their own implementation of FunctionsBase - /// - /// - /// - /// - public class SimpleSessionFunctions : SessionFunctionsBase - { - private readonly Func merger; - public SimpleSessionFunctions() => merger = (l, r) => l; - public SimpleSessionFunctions(Func merger) => this.merger = merger; + public virtual bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo) => true; /// - public override bool ConcurrentReader(ref TKey key, ref TValue input, ref TValue value, ref TValue dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public virtual bool InitialDeleter(ref LogRecord dstLogRecord, ref DeleteInfo deleteInfo) { - dst = value; + if (dstLogRecord.Info.ValueIsObject) + dstLogRecord.ClearValueObject(_ => { }); return true; } + public virtual void PostInitialDeleter(ref LogRecord dstLogRecord, ref DeleteInfo deleteInfo) { } + public virtual bool InPlaceDeleter(ref LogRecord dstLogRecord, ref DeleteInfo deleteInfo) => true; + public virtual void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } /// - public override bool SingleReader(ref TKey key, ref TValue input, ref TValue value, ref TValue dst, ref ReadInfo readInfo) - { - dst = value; - return true; - } - - public override bool SingleWriter(ref TKey key, ref TValue input, ref TValue src, ref TValue dst, ref TValue output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - var result = base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); - if (result) - output = dst; - return result; - } - - public override bool ConcurrentWriter(ref TKey key, ref TValue input, ref TValue src, ref TValue dst, ref TValue output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - var result = base.ConcurrentWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo); - if (result) - output = dst; - return result; - } + public virtual void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref TInput input, ref TOutput output, TContext ctx, Status status, RecordMetadata recordMetadata) { } + // *FieldInfo require an implementation that knows what is in IInput /// - public override bool InitialUpdater(ref TKey key, ref TValue input, ref TValue value, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) { value = output = input; return true; } + public virtual RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord + => throw new NotImplementedException("GetRMWModifiedFieldInfo requires knowledge of TInput"); /// - public override bool CopyUpdater(ref TKey key, ref TValue input, ref TValue oldValue, ref TValue newValue, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) { newValue = output = merger(input, oldValue); return true; } + public virtual RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TInput input) => throw new NotImplementedException("GetRMWInitialFieldInfo requires knowledge of TInput"); /// - public override bool InPlaceUpdater(ref TKey key, ref TValue input, ref TValue value, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) { value = output = merger(input, value); return true; } - } + public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref TInput input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length, ValueIsObject = false }; + /// + public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input) + => new () { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + /// + public virtual RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref 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 }; - public class SimpleSimpleFunctions : SimpleSessionFunctions - { - public SimpleSimpleFunctions() : base() { } - public SimpleSimpleFunctions(Func merger) : base(merger) { } + /// + public virtual void ConvertOutputToHeap(ref TInput input, ref TOutput output) { } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/TryAddFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/TryAddFunctions.cs deleted file mode 100644 index fb40ef86555..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/TryAddFunctions.cs +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - /// - /// Functions that make RMW behave as an atomic TryAdd operation, where Input is the value being added. - /// Return Status.NotFound => TryAdd succeededed (item added). - /// Return Status.Found => TryAdd failed (item not added, key was already present). - /// - /// - /// - /// - public class TryAddFunctions : SimpleSessionFunctions - { - /// - public override bool InPlaceUpdater(ref TKey key, ref TValue input, ref TValue value, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => true; - /// - public override bool NeedCopyUpdate(ref TKey key, ref TValue input, ref TValue oldValue, ref TValue output, ref RMWInfo rmwInfo) => false; - } - - /// - /// Functions that make RMW behave as an atomic TryAdd operation, where Input is the value being added. - /// Return Status.NotFound => TryAdd succeededed (item added) - /// Return Status.Found => TryAdd failed (item not added, key was already present) - /// - /// - /// - public class TryAddFunctions : TryAddFunctions { } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs index e5ca2cc6b6f..d7923144649 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs @@ -27,9 +27,9 @@ internal static class EpochPhaseIdx public const int CheckpointCompletionCallback = 4; } - public partial class TsavoriteKV - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal TaskCompletionSource checkpointTcs = new(TaskCreationOptions.RunContinuationsAsynchronously); @@ -100,12 +100,6 @@ internal void InitializeHybridLogCheckpoint(Guid hybridLogToken, long version) _hybridLogCheckpoint.Initialize(hybridLogToken, version, checkpointManager); } - internal long Compact(ISessionFunctions functions, CompactionFunctions compactionFunctions, long untilAddress, CompactionType compactionType) - where CompactionFunctions : ICompactionFunctions - { - throw new NotImplementedException(); - } - // #endregion } } \ No newline at end of file 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 9890d791ad2..3a4439b6592 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/DeltaLog.cs @@ -75,7 +75,7 @@ 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), ScanBufferingMode.SinglePageBuffering, false, default, logPageSizeBits, false, logger: logger) + : base(0, tailAddress >= 0 ? tailAddress : deltaLogDevice.GetFileSize(0), DiskScanBufferingMode.SinglePageBuffering, InMemoryScanBufferingMode.NoBuffering, includeSealedRecords: false, default, logPageSizeBits, false, logger: logger) { LogPageSizeBits = logPageSizeBits; PageSize = 1 << LogPageSizeBits; @@ -125,13 +125,10 @@ internal override void AsyncReadPagesFromDeviceToFrame(long readPageSt { int pageIndex = (int)(readPage % frame.frameSize); if (frame.frame[pageIndex] == null) - { frame.Allocate(pageIndex); - } else - { frame.Clear(pageIndex); - } + var asyncResult = new PageAsyncReadResult() { page = readPage, @@ -141,7 +138,6 @@ internal override void AsyncReadPagesFromDeviceToFrame(long readPageSt }; ulong offsetInFile = (ulong)(AlignedPageSizeBytes * readPage); - uint readLength = (uint)AlignedPageSizeBytes; long adjustedUntilAddress = (AlignedPageSizeBytes * (untilAddress >> LogPageSizeBits) + (untilAddress & PageSizeMask)); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs index ea0400d606a..a2d78b88694 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs @@ -11,9 +11,9 @@ namespace Tsavorite.core { internal unsafe delegate void SkipReadCache(HashBucket* bucket); - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Derived class facing persistence API internal IndexCheckpointInfo _indexCheckpoint; 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 10798bcf527..0287a86840c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs @@ -157,9 +157,9 @@ public struct LogFileInfo public long deltaLogTailAddress; } - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { private const long NoPageFreed = -1; @@ -774,7 +774,7 @@ private long RecoverHybridLog(long scanFromAddress, long recoverFromAddress, lon lastFreedPage = freedPage; // We make an extra pass to clear locks when reading every page back into memory - ClearLocksOnPage(p, options); + ClearBitsOnPage(p, options); ProcessReadPageAndFlush(recoverFromAddress, untilAddress, nextVersion, options, recoveryStatus, p, pageIndex); } @@ -807,7 +807,7 @@ private async ValueTask RecoverHybridLogAsync(long scanFromAddress, long r lastFreedPage = freedPage; // We make an extra pass to clear locks when reading every page back into memory - ClearLocksOnPage(p, options); + ClearBitsOnPage(p, options); ProcessReadPageAndFlush(recoverFromAddress, untilAddress, nextVersion, options, recoveryStatus, p, pageIndex); } @@ -910,7 +910,7 @@ private long RecoverHybridLogFromSnapshotFile(long scanFromAddress, long recover lastFreedPage = freedPage; // We make an extra pass to clear locks when reading pages back into memory - ClearLocksOnPage(p, options); + ClearBitsOnPage(p, options); } else { @@ -952,7 +952,7 @@ private async ValueTask RecoverHybridLogFromSnapshotFileAsync(long scanFro lastFreedPage = freedPage; // We make an extra pass to clear locks when reading pages back into memory - ClearLocksOnPage(p, options); + ClearBitsOnPage(p, options); } else { @@ -1062,7 +1062,7 @@ private void ProcessReadSnapshotPage(long fromAddress, long untilAddress, long n recoveryStatus.flushStatus[pageIndex] = FlushStatus.Done; } - private unsafe void ClearLocksOnPage(long page, RecoveryOptions options) + private unsafe void ClearBitsOnPage(long page, RecoveryOptions options) { var startLogicalAddress = hlog.GetStartLogicalAddress(page); var endLogicalAddress = hlog.GetStartLogicalAddress(page + 1); @@ -1077,15 +1077,16 @@ private unsafe void ClearLocksOnPage(long page, RecoveryOptions options) while (pointer < untilLogicalAddressInPage) { long recordStart = physicalAddress + pointer; - ref RecordInfo info = ref hlog.GetInfo(recordStart); - info.ClearBitsForDiskImages(); - if (info.IsNull()) + // DiskLogRecord ctor calls ClearBitsForDiskImages(), and then we use its size to move to the next record. + var diskLogRecord = new DiskLogRecord(recordStart); + + if (diskLogRecord.Info.IsNull) pointer += RecordInfo.GetLength(); else { - int size = hlog.GetRecordSize(recordStart).Item2; - Debug.Assert(size <= hlogBase.GetPageSize()); + 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 pointer += size; } } @@ -1110,9 +1111,10 @@ private unsafe bool RecoverFromPage(long startRecoveryAddress, while (pointer < untilLogicalAddressInPage) { recordStart = pagePhysicalAddress + pointer; - ref RecordInfo info = ref hlog.GetInfo(recordStart); + var diskLogRecord = new DiskLogRecord(recordStart); + ref RecordInfo info = ref diskLogRecord.InfoRef; - if (info.IsNull()) + if (info.IsNull) { pointer += RecordInfo.GetLength(); continue; @@ -1120,7 +1122,7 @@ private unsafe bool RecoverFromPage(long startRecoveryAddress, if (!info.Invalid) { - HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(ref hlog.GetKey(recordStart))); + HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(diskLogRecord.Key)); FindOrCreateTag(ref hei, hlogBase.BeginAddress); bool ignoreRecord = ((pageLogicalAddress + pointer) >= options.fuzzyRegionStartAddress) && info.IsInNewVersion; @@ -1146,7 +1148,7 @@ private unsafe bool RecoverFromPage(long startRecoveryAddress, } } } - pointer += hlog.GetRecordSize(recordStart).Item2; + pointer += diskLogRecord.GetSerializedLength(); } return touched; @@ -1176,9 +1178,9 @@ private void AsyncFlushPageCallbackForRecovery(uint errorCode, uint numBytes, ob } } - public abstract partial class AllocatorBase : IDisposable - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public abstract partial class AllocatorBase : IDisposable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Restore log 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 79edca27949..082eb5bf867 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/DisposeReason.cs @@ -4,7 +4,7 @@ namespace Tsavorite.core { /// - /// The reason for a call to + /// The reason for a call to /// public enum DisposeReason { @@ -14,9 +14,14 @@ public enum DisposeReason None, /// - /// Failure of SingleWriter insertion of a record at the tail of the cache. + /// CopyUpdate cleared the object immediately for more efficient size tracking /// - SingleWriterCASFailed, + CopyUpdated, + + /// + /// Failure of InitialWriter insertion of a record at the tail of the cache. + /// + InitialWriterCASFailed, /// /// Failure of CopyUpdater insertion of a record at the tail of the cache. @@ -29,9 +34,14 @@ public enum DisposeReason InitialUpdaterCASFailed, /// - /// Failure of SingleDeleter insertion of a record at the tail of the cache. + /// Failure of InitialDeleter insertion of a record at the tail of the cache. + /// + InitialDeleterCASFailed, + + /// + /// Some CAS failed and retry could not use the record due to size or address restrictions /// - SingleDeleterCASFailed, + CASAndRetryFailed, /// /// A record was deserialized from the disk for a pending Read or RMW operation. @@ -43,6 +53,16 @@ public enum DisposeReason /// RevivificationFreeList, + /// + /// Deleted but remains in hash chain so Key is unchanged + /// + Deleted, + + /// + /// Elided from hash chain but not put into Revivification free list + /// + Elided, + /// /// A page was evicted from the in-memory portion of the main log, or from the readcache. /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IRecordDisposer.cs b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IRecordDisposer.cs index 496fe93dce4..e8a562ffe96 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IRecordDisposer.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IRecordDisposer.cs @@ -1,38 +1,37 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. -using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { /// - /// Interface to implement the Disposer component of + /// Interface to implement the Disposer component of /// - public interface IRecordDisposer + public interface IRecordDisposer { /// - /// If true, with + /// If true, with /// is called on page evictions from both readcache and main log. Otherwise, the user can register an Observer and do any needed disposal there. /// public bool DisposeOnPageEviction { get; } /// - /// Dispose the Key and Value of a record, if necessary. See comments in for details. + /// Dispose the Key and Value of a record, if necessary. See comments in for details. /// - void DisposeRecord(ref TKey key, ref TValue value, DisposeReason reason, int newKeySize); + void DisposeValueObject(IHeapObject valueObject, DisposeReason reason); } /// - /// Default no-op implementation if + /// Default no-op implementation if /// /// It is appropriate to call methods on this instance as a no-op. - public struct DefaultRecordDisposer : IRecordDisposer + public struct DefaultRecordDisposer : IRecordDisposer { /// /// Default instance /// - public static readonly DefaultRecordDisposer Instance = new(); + public static readonly DefaultRecordDisposer Instance = new(); /// /// Assumes the key and value have no need of Dispose(), and does nothing. @@ -42,16 +41,13 @@ public struct DefaultRecordDisposer : IRecordDisposer /// Assumes the key and value have no need of Dispose(), and does nothing. /// - public readonly void DisposeRecord(ref TKey key, ref TValue value, DisposeReason reason, int newKeySize) - { - Debug.Assert(typeof(TKey) != typeof(SpanByte) && typeof(TValue) != typeof(SpanByte), "Must use SpanByteRecordDisposer"); - } + public readonly void DisposeValueObject(IHeapObject valueObject, DisposeReason reason) { } } /// - /// Default no-op implementation if for SpanByte + /// No-op implementation of for SpanByte /// - public struct SpanByteRecordDisposer : IRecordDisposer + public struct SpanByteRecordDisposer : IRecordDisposer // TODO remove for dual { /// /// Default instance @@ -63,44 +59,8 @@ public struct SpanByteRecordDisposer : IRecordDisposer /// public readonly bool DisposeOnPageEviction => false; - /// - /// If is and is >= 0, - /// this adjusts the key (and if necessary value) space as needed to preserve log zero-init correctness. - /// Otherwise the key and value have no need of disposal, and this does nothing. - /// + /// No-op implementation because SpanByte values have no need for disposal. [MethodImpl(MethodImplOptions.AggressiveInlining)] - public unsafe void DisposeRecord(ref SpanByte key, ref SpanByte value, DisposeReason reason, int newKeySize) - { - // We don't have to do anything with the Value unless the new key size requires adjusting the key length. - // newKeySize == -1 means we are preserving the existing key (e.g. for in-chain revivification). - if (reason != DisposeReason.RevivificationFreeList || newKeySize < 0) - return; - - var oldKeySize = Utility.RoundUp(key.TotalSize, Constants.kRecordAlignment); - - // We are changing the key size (e.g. revivification from the freelist with a new key). - // Our math here uses record alignment of keys as in the allocator, and assumes this will always be at least int alignment. - newKeySize = Utility.RoundUp(newKeySize, Constants.kRecordAlignment); - int keySizeChange = newKeySize - oldKeySize; - if (keySizeChange == 0) - return; - - // We are growing or shrinking. We don't care (here or in SingleWriter, InitialUpdater, CopyUpdater) what is inside the Key and Value, - // as long as we don't leave nonzero bytes after the used value space. So we just need to make sure the Value space starts immediately - // after the new key size. SingleWriter et al. will do the ShrinkSerializedLength on Value as needed. - if (keySizeChange < 0) - { - // We are shrinking the key; the Value of the new record will start after key + newKeySize, so set the new value length there. - *(int*)((byte*)Unsafe.AsPointer(ref key) + newKeySize) = value.Length - keySizeChange; // minus negative => plus positive - } - else - { - // We are growing the key; the Value of the new record will start somewhere in the middle of where the old Value was, so set the new value length there. - *(int*)((byte*)Unsafe.AsPointer(ref value) + keySizeChange) = value.Length - keySizeChange; - } - - // NewKeySize is (newKey).TotalSize. - key.Length = newKeySize - sizeof(int); - } + public unsafe void DisposeValueObject(IHeapObject valueObject, DisposeReason reason) { } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IStoreFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IStoreFunctions.cs index 5d2245620d1..8da16f1539d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IStoreFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/IStoreFunctions.cs @@ -9,56 +9,42 @@ namespace Tsavorite.core /// /// The interface to define functions on the TsavoriteKV store itself (rather than a session). /// - public interface IStoreFunctions + public interface IStoreFunctions { #region Key Comparer /// Get a 64-bit hash code for a key - long GetKeyHashCode64(ref TKey key); + long GetKeyHashCode64(ReadOnlySpan key); /// Compare two keys for equality - bool KeysEqual(ref TKey k1, ref TKey k2); + bool KeysEqual(ReadOnlySpan k1, ReadOnlySpan k2); #endregion Key Comparer - #region Key Serializer - /// Indicates whether the Key Serializer is to be used - bool HasKeySerializer { get; } - - /// Instatiate a KeySerializer and begin Key serialization to the given stream. - /// This must instantiate a new serializer as multiple threads may be serializing or deserializing. - IObjectSerializer BeginSerializeKey(Stream stream); - - /// Instatiate a KeySerializer and begin Key deserialization from the given stream. - /// This must instantiate a new serializer as multiple threads may be serializing or deserializing. - IObjectSerializer BeginDeserializeKey(Stream stream); - #endregion Key Serializer - #region Value Serializer + /// Creates an instance of the Value Serializer + IObjectSerializer CreateValueObjectSerializer(); + /// Indicates whether the Value Serializer is to be used bool HasValueSerializer { get; } /// Instatiate a ValueSerializer and begin Value serialization to the given stream. /// This must instantiate a new serializer as multiple threads may be serializing or deserializing. - IObjectSerializer BeginSerializeValue(Stream stream); + IObjectSerializer BeginSerializeValue(Stream stream); /// Instatiate a ValueSerializer and begin Value deserialization from the given stream. /// This must instantiate a new serializer as multiple threads may be serializing or deserializing. - IObjectSerializer BeginDeserializeValue(Stream stream); + IObjectSerializer BeginDeserializeValue(Stream stream); #endregion Value Serializer #region Record Disposer /// - /// If true, with + /// If true, with /// is called on page evictions from both readcache and main log. Otherwise, the user can register an Observer and /// do any needed disposal there. /// bool DisposeOnPageEviction { get; } - /// Dispose the Key and Value of a record, if necessary. - /// The key for the record - /// The value for the record - /// For only, this is a record from the freelist and we may be disposing the key as well as value - /// (it is -1 when revivifying a record in the hash chain or when doing a RETRY; for these the key does not change) - void DisposeRecord(ref TKey key, ref TValue value, DisposeReason reason, int newKeySize = -1); + /// Dispose the Value of a record, if necessary. + void DisposeValueObject(IHeapObject valueObject, DisposeReason reason); #endregion Record Disposer #region Checkpoint Completion 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 a920c2f6463..88d4624bd60 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/StoreFunctions/StoreFunctions.cs @@ -8,7 +8,7 @@ namespace Tsavorite.core { /// - /// Store functions for and . + /// Store functions for an instance of TsavoriteKV. /// /// /// The implementation takes instances of the supported interfaces (e.g. ) to allow custom @@ -16,21 +16,16 @@ namespace Tsavorite.core /// because there is no need to wrap calls to them with additional functionality. This can be changed to redirect if such wrapper /// functionality is needed. /// - public struct StoreFunctions - (TKeyComparer keyComparer, Func> keySerializerCreator, Func> valueSerializerCreator, TRecordDisposer recordDisposer) - : IStoreFunctions - where TKeyComparer : IKeyComparer - where TRecordDisposer : IRecordDisposer + public struct StoreFunctions(TKeyComparer keyComparer, Func> valueSerializerCreator, TRecordDisposer recordDisposer) : IStoreFunctions + where TKeyComparer : IKeyComparer + where TRecordDisposer : IRecordDisposer { #region Fields /// Compare two keys for equality, and get a key's hash code. readonly TKeyComparer keyComparer = keyComparer; - /// Serialize a Key to persistent storage - readonly Func> keySerializerCreator = keySerializerCreator; - /// Serialize a Value to persistent storage - readonly Func> valueSerializerCreator = valueSerializerCreator; + readonly Func> valueSerializerCreator = valueSerializerCreator; /// Dispose a record readonly TRecordDisposer recordDisposer = recordDisposer; @@ -42,50 +37,32 @@ public struct StoreFunctions #region Key Comparer /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly long GetKeyHashCode64(ref TKey key) => keyComparer.GetHashCode64(ref key); + public readonly long GetKeyHashCode64(ReadOnlySpan key) => keyComparer.GetHashCode64(key); /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly bool KeysEqual(ref TKey k1, ref TKey k2) => keyComparer.Equals(ref k1, ref k2); + public readonly bool KeysEqual(ReadOnlySpan k1, ReadOnlySpan k2) => keyComparer.Equals(k1, k2); #endregion Key Comparer - #region Key Serializer - /// - public readonly bool HasKeySerializer => keySerializerCreator is not null; - - /// - public readonly IObjectSerializer BeginSerializeKey(Stream stream) - { - var keySerializer = keySerializerCreator(); - keySerializer.BeginSerialize(stream); - return keySerializer; - } - + #region Value Serializer /// - public readonly IObjectSerializer BeginDeserializeKey(Stream stream) - { - var keySerializer = keySerializerCreator(); - keySerializer.BeginDeserialize(stream); - return keySerializer; - } - #endregion Key Serializer + public readonly IObjectSerializer CreateValueObjectSerializer() => valueSerializerCreator(); - #region Value Serializer /// public readonly bool HasValueSerializer => valueSerializerCreator is not null; /// - public readonly IObjectSerializer BeginSerializeValue(Stream stream) + public readonly IObjectSerializer BeginSerializeValue(Stream stream) { - var valueSerializer = valueSerializerCreator(); + var valueSerializer = CreateValueObjectSerializer(); valueSerializer.BeginSerialize(stream); return valueSerializer; } /// - public readonly IObjectSerializer BeginDeserializeValue(Stream stream) + public readonly IObjectSerializer BeginDeserializeValue(Stream stream) { - var valueSerializer = valueSerializerCreator(); + var valueSerializer = CreateValueObjectSerializer(); valueSerializer.BeginDeserialize(stream); return valueSerializer; } @@ -97,7 +74,7 @@ public readonly IObjectSerializer BeginDeserializeValue(Stream stream) /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly void DisposeRecord(ref TKey key, ref TValue value, DisposeReason reason, int newKeySize) => recordDisposer.DisposeRecord(ref key, ref value, reason, newKeySize); + public readonly void DisposeValueObject(IHeapObject valueObject, DisposeReason reason) => recordDisposer.DisposeValueObject(valueObject, reason); #endregion Record Disposer #region Checkpoint Completion @@ -110,48 +87,45 @@ public readonly IObjectSerializer BeginDeserializeValue(Stream stream) } /// - /// A non-parameterized version of StoreFunctions that provides type-reduced Create() methods. + /// A minimally-parameterized version of StoreFunctions that provides type-reduced Create() methods. /// - public struct StoreFunctions + public struct StoreFunctions { /// /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. /// - public static StoreFunctions Create - (TKeyComparer keyComparer, Func> keySerializerCreator, Func> valueSerializerCreator, TRecordDisposer recordDisposer) - where TKeyComparer : IKeyComparer - where TRecordDisposer : IRecordDisposer - => new(keyComparer, keySerializerCreator, valueSerializerCreator, recordDisposer); + public static StoreFunctions Create + (TKeyComparer keyComparer, Func> valueSerializerCreator, TRecordDisposer recordDisposer) + where TKeyComparer : IKeyComparer + where TRecordDisposer : IRecordDisposer + => new(keyComparer, valueSerializerCreator, recordDisposer); /// /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. /// - public static StoreFunctions> Create - (TKeyComparer keyComparer, Func> keySerializerCreator, Func> valueSerializerCreator) - where TKeyComparer : IKeyComparer - => new(keyComparer, keySerializerCreator, valueSerializerCreator, new DefaultRecordDisposer()); + public static StoreFunctions Create(TKeyComparer keyComparer, Func> valueSerializerCreator) + where TKeyComparer : IKeyComparer + => new(keyComparer, valueSerializerCreator, new DefaultRecordDisposer()); /// /// Construct a StoreFunctions instance with all types specified and contained instances passed, e.g. for custom objects. /// - public static StoreFunctions Create - (TKeyComparer keyComparer, TRecordDisposer recordDisposer) - where TKeyComparer : IKeyComparer - where TRecordDisposer : IRecordDisposer - => new(keyComparer, keySerializerCreator: null, valueSerializerCreator: null, recordDisposer); + public static StoreFunctions Create(TKeyComparer keyComparer, TRecordDisposer recordDisposer) + where TKeyComparer : IKeyComparer + where TRecordDisposer : IRecordDisposer + => new(keyComparer, valueSerializerCreator: null, recordDisposer); /// - /// Store functions for and that take only the + /// Store functions that take only the /// - public static StoreFunctions> Create - (TKeyComparer keyComparer) - where TKeyComparer : IKeyComparer - => new(keyComparer, keySerializerCreator: null, valueSerializerCreator: null, DefaultRecordDisposer.Instance); + public static StoreFunctions Create(TKeyComparer keyComparer) + where TKeyComparer : IKeyComparer + => new(keyComparer, valueSerializerCreator: null, DefaultRecordDisposer.Instance); /// - /// Store functions for Key and Value + /// Store functions for Key and Value /// - public static StoreFunctions Create() - => new(SpanByteComparer.Instance, keySerializerCreator: null, valueSerializerCreator: null, SpanByteRecordDisposer.Instance); + public static StoreFunctions Create() + => new(SpanByteComparer.Instance, valueSerializerCreator: null, SpanByteRecordDisposer.Instance); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Extensions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Extensions.cs deleted file mode 100644 index f3e6079939e..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Extensions.cs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -#pragma warning disable 0162 - -using System; - -namespace Tsavorite.core -{ - /// - /// Log subscription extensions - /// - public static class Extensions - { - /// - /// Create observable of log records - /// - /// - /// - /// - /// - public static IObservable> ToRecordObservable(this IObservable> source) - { - return new RecordObservable(source); - } - - internal sealed class RecordObservable : IObservable> - { - readonly IObservable> o; - - public RecordObservable(IObservable> o) - { - this.o = o; - } - - public IDisposable Subscribe(IObserver> observer) - { - return o.Subscribe(new RecordObserver(observer)); - } - } - - internal sealed class RecordObserver : IObserver> - { - private readonly IObserver> observer; - - public RecordObserver(IObserver> observer) - { - this.observer = observer; - } - - public void OnCompleted() - { - observer.OnCompleted(); - } - - public void OnError(Exception error) - { - observer.OnError(error); - } - - public void OnNext(ITsavoriteScanIterator v) - { - while (v.GetNext(out RecordInfo info, out TKey key, out TValue value)) - { - observer.OnNext(new AllocatorRecord { info = info, key = key, value = value }); - } - } - } - } -} \ No newline at end of file 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 9b0173c6ab5..f2474584895 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 @@ -7,13 +7,13 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] private static bool TryBlockAllocate( - AllocatorBase allocator, + AllocatorBase allocator, int recordSize, out long logicalAddress, ref PendingContext pendingContext, @@ -39,17 +39,17 @@ private static bool TryBlockAllocate( internal struct AllocateOptions { /// If true, use the non-revivification recycling of records that failed to CAS and are carried in PendingContext through RETRY. - internal bool Recycle; + internal bool recycle; /// If true, the source record is elidable so we can try to elide from the tag chain (and transfer it to the FreeList if we're doing Revivification). - internal bool ElideSourceRecord; + internal bool elideSourceRecord; }; [MethodImpl(MethodImplOptions.AggressiveInlining)] bool TryAllocateRecord(TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext, - ref OperationStackContext stackCtx, int actualSize, ref int allocatedSize, int newKeySize, AllocateOptions options, - out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, ref RecordSizeInfo sizeInfo, AllocateOptions options, + out long newLogicalAddress, out long newPhysicalAddress, out int allocatedSize, out OperationStatus status) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { status = OperationStatus.SUCCESS; @@ -57,11 +57,15 @@ bool TryAllocateRecord(TSes var minMutableAddress = GetMinRevivifiableAddress(); var minRevivAddress = minMutableAddress; - if (options.Recycle && pendingContext.retryNewLogicalAddress != Constants.kInvalidAddress && GetAllocationForRetry(sessionFunctions, ref pendingContext, minRevivAddress, ref allocatedSize, newKeySize, out newLogicalAddress, out newPhysicalAddress)) + if (options.recycle && pendingContext.retryNewLogicalAddress != Constants.kInvalidAddress + && GetAllocationForRetry(sessionFunctions, ref pendingContext, minRevivAddress, ref sizeInfo, out newLogicalAddress, out newPhysicalAddress, out allocatedSize)) + { + new LogRecord(newPhysicalAddress).PrepareForRevivification(ref sizeInfo, allocatedSize); return true; + } if (RevivificationManager.UseFreeRecordPool) { - if (!options.ElideSourceRecord && stackCtx.hei.Address >= minMutableAddress) + if (!options.elideSourceRecord && stackCtx.hei.Address >= minMutableAddress) minRevivAddress = stackCtx.hei.Address; if (sessionFunctions.Ctx.IsInV1) { @@ -69,63 +73,76 @@ bool TryAllocateRecord(TSes if (fuzzyStartAddress > minRevivAddress) minRevivAddress = fuzzyStartAddress; } - if (TryTakeFreeRecord(sessionFunctions, actualSize, ref allocatedSize, newKeySize, minRevivAddress, out newLogicalAddress, out newPhysicalAddress)) + if (TryTakeFreeRecord(sessionFunctions, ref sizeInfo, minRevivAddress, out newLogicalAddress, out newPhysicalAddress, out allocatedSize)) + { + new LogRecord(newPhysicalAddress).PrepareForRevivification(ref sizeInfo, allocatedSize); return true; + } } // Spin to make sure newLogicalAddress is > recSrc.LatestLogicalAddress (the .PreviousAddress and CAS comparison value). - for (; ; Thread.Yield()) + for (; ; _ = Thread.Yield()) { - if (!TryBlockAllocate(hlogBase, allocatedSize, out newLogicalAddress, ref pendingContext, out status)) + if (!TryBlockAllocate(hlogBase, sizeInfo.AllocatedInlineRecordSize, out newLogicalAddress, ref pendingContext, out status)) break; newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); if (VerifyInMemoryAddresses(ref stackCtx)) { + allocatedSize = sizeInfo.AllocatedInlineRecordSize; if (newLogicalAddress > stackCtx.recSrc.LatestLogicalAddress) return true; // This allocation is below the necessary address so put it on the free list or abandon it, then repeat the loop. - if (!RevivificationManager.UseFreeRecordPool || !RevivificationManager.TryAdd(newLogicalAddress, newPhysicalAddress, allocatedSize, ref sessionFunctions.Ctx.RevivificationStats)) - hlog.GetInfo(newPhysicalAddress).SetInvalid(); // Skip on log scan + if (RevivificationManager.UseFreeRecordPool) + { + // Set up a simple LogRecord with specified key size and value size taking the entire non-key space (we don't have optionals now) + // so revivification can read the record size. + var logRecord = hlog.CreateLogRecord(newLogicalAddress, newPhysicalAddress); + logRecord.InitializeForReuse(ref sizeInfo); + if (RevivificationManager.TryAdd(newLogicalAddress, ref logRecord, ref sessionFunctions.Ctx.RevivificationStats)) + continue; + } + LogRecord.GetInfo(newPhysicalAddress).SetInvalid(); // Skip on log scan continue; } // In-memory source dropped below HeadAddress during BlockAllocate. Save the record for retry if we can. - ref var newRecordInfo = ref hlog.GetInfo(newPhysicalAddress); - if (options.Recycle) + if (options.recycle) { - ref var newValue = ref hlog.GetValue(newPhysicalAddress); - _ = hlog.GetAndInitializeValue(newPhysicalAddress, newPhysicalAddress + actualSize); - var valueOffset = (int)((long)Unsafe.AsPointer(ref newValue) - newPhysicalAddress); - SetExtraValueLength(ref hlog.GetValue(newPhysicalAddress), ref newRecordInfo, actualSize - valueOffset, allocatedSize - valueOffset); - SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress, allocatedSize); + var logRecord = new LogRecord(newPhysicalAddress); + logRecord.InitializeForReuse(ref sizeInfo); + SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress); } else - newRecordInfo.SetInvalid(); // Skip on log scan + LogRecord.GetInfoRef(newPhysicalAddress).SetInvalid(); // Skip on log scan status = OperationStatus.RETRY_LATER; break; } newPhysicalAddress = 0; + allocatedSize = 0; return false; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool TryAllocateRecordReadCache(ref PendingContext pendingContext, ref OperationStackContext stackCtx, - int allocatedSize, out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status) + bool TryAllocateRecordReadCache(ref PendingContext pendingContext, ref OperationStackContext stackCtx, + ref RecordSizeInfo recordSizeInfo, out long newLogicalAddress, out long newPhysicalAddress, out int allocatedSize, 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. for (; ; Thread.Yield()) { - if (!TryBlockAllocate(readCacheBase, allocatedSize, out newLogicalAddress, ref pendingContext, out status)) + if (!TryBlockAllocate(readCacheBase, recordSizeInfo.AllocatedInlineRecordSize, out newLogicalAddress, ref pendingContext, out status)) break; newPhysicalAddress = readcache.GetPhysicalAddress(newLogicalAddress); if (VerifyInMemoryAddresses(ref stackCtx)) { if (!stackCtx.hei.IsReadCache || newLogicalAddress > stackCtx.hei.AbsoluteAddress) + { + allocatedSize = recordSizeInfo.AllocatedInlineRecordSize; return true; + } // This allocation is below the necessary address so abandon it and repeat the loop. ReadCacheAbandonRecord(newPhysicalAddress); @@ -139,13 +156,14 @@ bool TryAllocateRecordReadCache(ref PendingContext(ref PendingContext pendingContext, long logicalAddress, long physicalAddress, int allocatedSize) + // Do not inline, to keep CreateNewRecord* lean + void SaveAllocationForRetry(ref PendingContext pendingContext, long logicalAddress, long physicalAddress) { - ref var recordInfo = ref hlog.GetInfo(physicalAddress); + ref var recordInfo = ref LogRecord.GetInfoRef(physicalAddress); // TryAllocateRecord may stash this before WriteRecordInfo is called, leaving .PreviousAddress set to kInvalidAddress. // This is zero, and setting Invalid will result in recordInfo.IsNull being true, which will cause log-scan problems. @@ -153,45 +171,38 @@ void SaveAllocationForRetry(ref PendingContext(TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext, long minAddress, - ref int allocatedSize, int newKeySize, out long newLogicalAddress, out long newPhysicalAddress) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref RecordSizeInfo sizeInfo, out long newLogicalAddress, out long newPhysicalAddress, out int allocatedSize) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // Use an earlier allocation from a failed operation, if possible. newLogicalAddress = pendingContext.retryNewLogicalAddress; pendingContext.retryNewLogicalAddress = 0; - if (newLogicalAddress < hlogBase.HeadAddress) + if (newLogicalAddress <= minAddress || newLogicalAddress < hlogBase.HeadAddress) { - // The record dropped below headAddress. If it needs DisposeRecord, it will be done on eviction. - newPhysicalAddress = 0; - return false; + // The record is too small or dropped below headAddress. + goto Fail; } newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); - ref var recordInfo = ref hlog.GetInfo(newPhysicalAddress); - Debug.Assert(!recordInfo.IsNull(), "RecordInfo should not be IsNull"); - ref var recordValue = ref hlog.GetValue(newPhysicalAddress); - (int usedValueLength, int fullValueLength, int fullRecordLength) = GetRecordLengths(newPhysicalAddress, ref recordValue, ref recordInfo); - - // Dispose the record for either reuse or abandonment. - ClearExtraValueSpace(ref recordInfo, ref recordValue, usedValueLength, fullValueLength); - storeFunctions.DisposeRecord(ref hlog.GetKey(newPhysicalAddress), ref recordValue, DisposeReason.RevivificationFreeList, newKeySize); + var newLogRecord = new LogRecord(newPhysicalAddress); - if (newLogicalAddress <= minAddress || fullRecordLength < allocatedSize) - { - // Can't reuse, so abandon it. - newPhysicalAddress = 0; - return false; - } - - allocatedSize = fullRecordLength; + allocatedSize = newLogRecord.GetInlineRecordSizes().allocatedSize; + if (allocatedSize < sizeInfo.AllocatedInlineRecordSize) + goto Fail; return true; + + Fail: + var logRecord = hlog.CreateLogRecord(newLogicalAddress); + DisposeRecord(ref logRecord, DisposeReason.CASAndRetryFailed); + allocatedSize = 0; + newPhysicalAddress = 0; + return false; } } } \ No newline at end of file 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 1c66b5bbb08..a6c528632f7 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 @@ -6,33 +6,29 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Copy a record to the tail of the log after caller has verified it does not exist within a specified range. /// /// Callback functions. /// pending context created when the operation goes pending. - /// key of the record. - /// input passed through. - /// the value to insert - /// Location to store output computed from input and value. - /// user context corresponding to operation used during completion callback. + /// key of the record. /// Contains information about the call context, record metadata, and so on - /// The reason the CopyToTail is being done /// Whether to do IO if the search must go below HeadAddress. ReadFromImmutable, for example, /// is just an optimization to avoid future IOs, so if we need an IO here we just defer them to the next Read(). + /// Maximum address for determining liveness, records after this address are not considered when checking validity. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - private OperationStatus ConditionalCopyToTail(TSessionFunctionsWrapper sessionFunctions, - ref PendingContext pendingContext, - ref TKey key, ref TInput input, ref TValue value, ref TOutput output, TContext userContext, - ref OperationStackContext stackCtx, WriteReason writeReason, bool wantIO = true, long maxAddress = long.MaxValue) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private OperationStatus ConditionalCopyToTail(TSessionFunctionsWrapper sessionFunctions, + ref PendingContext pendingContext, ref TSourceLogRecord srcLogRecord, + ref OperationStackContext stackCtx, bool wantIO = true, long maxAddress = long.MaxValue) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - bool callerHasTransientLock = stackCtx.recSrc.HasTransientSLock; + var callerHasEphemeralLock = stackCtx.recSrc.HasEphemeralSLock; // We are called by one of ReadFromImmutable, CompactionConditionalCopyToTail, or ContinuePendingConditionalCopyToTail; // these have already searched to see if the record is present above minAddress, and stackCtx is set up for the first try. @@ -43,18 +39,17 @@ private OperationStatus ConditionalCopyToTail(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (callerHasEphemeralLock || TryEphemeralSLock(sessionFunctions, ref stackCtx, out var status)) { try { - RecordInfo dummyRecordInfo = default; // TryCopyToTail only needs this for readcache record invalidation. - status = TryCopyToTail(ref pendingContext, ref key, ref input, ref value, ref output, ref stackCtx, ref dummyRecordInfo, sessionFunctions, writeReason); + status = TryCopyToTail(ref srcLogRecord, sessionFunctions, ref pendingContext, ref stackCtx); } finally { stackCtx.HandleNewRecordOnException(this); - if (!callerHasTransientLock) - TransientSUnlock(sessionFunctions, ref key, ref stackCtx); + if (!callerHasEphemeralLock) + EphemeralSUnlock(sessionFunctions, ref stackCtx); } } @@ -66,11 +61,12 @@ private OperationStatus ConditionalCopyToTail stackCtx2 = new(stackCtx.hei.hash); + OperationStackContext stackCtx2 = new(stackCtx.hei.hash); bool needIO; do { - if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx2, stackCtx.recSrc.LogicalAddress, minAddress, maxAddress, out status, out needIO)) + if (TryFindRecordInMainLogForConditionalOperation( + sessionFunctions, srcLogRecord.Key, ref stackCtx2, stackCtx.recSrc.LogicalAddress, minAddress, maxAddress, out status, out needIO)) return OperationStatus.SUCCESS; } while (HandleImmediateNonPendingRetryStatus(status, sessionFunctions)); @@ -83,65 +79,51 @@ private OperationStatus ConditionalCopyToTail(TSessionFunctionsWrapper sessionFunctions, ref TKey key, ref TInput input, ref TValue value, - ref TOutput output, long currentAddress, long minAddress, long maxAddress = long.MaxValue) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status CompactionConditionalCopyToTail( + TSessionFunctionsWrapper sessionFunctions, ref TSourceLogRecord srcLogRecord, long currentAddress, long minAddress, long maxAddress = long.MaxValue) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { Debug.Assert(epoch.ThisInstanceProtected(), "This is called only from Compaction so the epoch should be protected"); PendingContext pendingContext = new(); - OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(ref key)); + OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(srcLogRecord.Key)); OperationStatus status; bool needIO; do { - if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx, currentAddress, minAddress, maxAddress, out status, out needIO)) + if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, srcLogRecord.Key, ref stackCtx, currentAddress, minAddress, maxAddress, out status, out needIO)) return Status.CreateFound(); } while (sessionFunctions.Store.HandleImmediateNonPendingRetryStatus(status, sessionFunctions)); if (needIO) - status = PrepareIOForConditionalOperation(sessionFunctions, ref pendingContext, ref key, ref input, ref value, ref output, default, - ref stackCtx, minAddress, maxAddress, WriteReason.Compaction); + status = PrepareIOForConditionalOperation(ref pendingContext, ref srcLogRecord, ref stackCtx, minAddress, maxAddress); else - status = ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref key, ref input, ref value, ref output, default, ref stackCtx, WriteReason.Compaction, true, maxAddress); + status = ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref srcLogRecord, ref stackCtx, maxAddress: maxAddress); return HandleOperationStatus(sessionFunctions.Ctx, ref pendingContext, status, out _); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus PrepareIOForConditionalOperation(TSessionFunctionsWrapper sessionFunctions, - ref PendingContext pendingContext, - ref TKey key, ref TInput input, ref TValue value, ref TOutput output, TContext userContext, - ref OperationStackContext stackCtx, long minAddress, long maxAddress, WriteReason writeReason, - OperationType opType = OperationType.CONDITIONAL_INSERT) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal OperationStatus PrepareIOForConditionalOperation( + ref PendingContext pendingContext, ref TSourceLogRecord srcLogRecord, + ref OperationStackContext stackCtx, long minAddress, long maxAddress, OperationType opType = OperationType.CONDITIONAL_INSERT) + where TSourceLogRecord : ISourceLogRecord { pendingContext.type = opType; pendingContext.minAddress = minAddress; pendingContext.maxAddress = maxAddress; - pendingContext.writeReason = writeReason; - pendingContext.InitialEntryAddress = Constants.kInvalidAddress; - pendingContext.InitialLatestLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; - - if (!pendingContext.NoKey && pendingContext.key == default) // If this is true, we don't have a valid key - pendingContext.key = hlog.GetKeyContainer(ref key); - if (pendingContext.input == default) - pendingContext.input = sessionFunctions.GetHeapContainer(ref input); - if (pendingContext.value == default) - pendingContext.value = hlog.GetValueContainer(ref value); - - pendingContext.output = output; - sessionFunctions.ConvertOutputToHeap(ref input, ref pendingContext.output); - - pendingContext.userContext = userContext; + pendingContext.initialEntryAddress = Constants.kInvalidAddress; + pendingContext.initialLatestLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; + if (!pendingContext.IsSet) + pendingContext.Serialize(ref srcLogRecord, hlogBase.bufferPool, valueSerializer: null); return OperationStatus.RECORD_ON_DISK; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContainsKeyInMemory.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContainsKeyInMemory.cs index d8c0c9c6876..69f15b46723 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContainsKeyInMemory.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContainsKeyInMemory.cs @@ -1,20 +1,21 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] internal Status InternalContainsKeyInMemory( - ref TKey key, TSessionFunctionsWrapper sessionFunctions, out long logicalAddress, long fromAddress = -1) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ReadOnlySpan key, TSessionFunctionsWrapper sessionFunctions, out long logicalAddress, long fromAddress = -1) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(ref key)); + OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(key)); if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); @@ -29,7 +30,7 @@ internal Status InternalContainsKeyInMemory : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Continue a pending read operation. Computes 'output' from 'input' and value corresponding to 'key' @@ -28,24 +28,24 @@ public unsafe partial class TsavoriteKV /// /// - internal OperationStatus ContinuePendingRead(AsyncIOContext request, + internal OperationStatus ContinuePendingRead(AsyncIOContext request, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - ref RecordInfo srcRecordInfo = ref hlog.GetInfoFromBytePointer(request.record.GetValidPointer()); - srcRecordInfo.ClearBitsForDiskImages(); + var diskRecord = new DiskLogRecord(ref request); + + if (pendingContext.IsReadAtAddress && !pendingContext.IsNoKey && !storeFunctions.KeysEqual(pendingContext.Key, diskRecord.Key)) + goto NotFound; if (request.logicalAddress >= hlogBase.BeginAddress && request.logicalAddress >= pendingContext.minAddress) { SpinWaitUntilClosed(request.logicalAddress); - // If NoKey, we do not have the key in the initial call and must use the key from the satisfied request. - ref TKey key = ref pendingContext.NoKey ? ref hlog.GetContextRecordKey(ref request) : ref pendingContext.key.Get(); - OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(ref key)); + OperationStackContext stackCtx = new(storeFunctions.GetKeyHashCode64(diskRecord.Key)); while (true) { - if (!FindTagAndTryTransientSLock(sessionFunctions, ref key, ref stackCtx, out var status)) + if (!FindTagAndTryEphemeralSLock(sessionFunctions, ref stackCtx, out var status)) { Debug.Assert(status != OperationStatus.NOTFOUND, "Expected to FindTag in InternalContinuePendingRead"); if (HandleImmediateRetryStatus(status, sessionFunctions, ref pendingContext)) @@ -57,19 +57,19 @@ internal OperationStatus ContinuePendingRead pendingContext.InitialLatestLogicalAddress + if (stackCtx.recSrc.LogicalAddress > pendingContext.initialLatestLogicalAddress && (!pendingContext.HasMinAddress || stackCtx.recSrc.LogicalAddress >= pendingContext.minAddress)) { OperationStatus internalStatus; do { - internalStatus = InternalRead(ref key, pendingContext.keyHash, ref pendingContext.input.Get(), ref pendingContext.output, + internalStatus = InternalRead(diskRecord.Key, pendingContext.keyHash, ref pendingContext.input.Get(), ref pendingContext.output, pendingContext.userContext, ref pendingContext, sessionFunctions); } while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pendingContext)); @@ -95,7 +95,7 @@ internal OperationStatus ContinuePendingRead= hlogBase.ReadOnlyAddress) { // If this succeeds, we don't need to copy to tail or readcache, so return success. - if (sessionFunctions.ConcurrentReader(ref key, ref pendingContext.input.Get(), ref value, ref pendingContext.output, ref readInfo, ref srcRecordInfo)) + if (sessionFunctions.Reader(ref memoryRecord, ref pendingContext.input.Get(), ref pendingContext.output, ref readInfo)) return OperationStatus.SUCCESS; } - else + else if (memoryRecord.IsSet) { // This may be in the immutable region, which means it may be an updated version of the record. - success = sessionFunctions.SingleReader(ref key, ref pendingContext.input.Get(), ref value, ref pendingContext.output, ref readInfo); + success = sessionFunctions.Reader(ref memoryRecord, ref pendingContext.input.Get(), ref pendingContext.output, ref readInfo); } + else + success = sessionFunctions.Reader(ref diskRecord, ref pendingContext.input.Get(), ref pendingContext.output, ref readInfo); if (!success) { - pendingContext.recordInfo = srcRecordInfo; if (readInfo.Action == ReadAction.CancelOperation) return OperationStatus.CANCELED; if (readInfo.Action == ReadAction.Expire) @@ -129,26 +129,24 @@ internal OperationStatus ContinuePendingRead(sessionFunctions, ref key, ref stackCtx); + EphemeralSUnlock(sessionFunctions, ref stackCtx); } // Must do this *after* Unlocking. Status was set by InternalTryCopyToTail. @@ -158,7 +156,6 @@ internal OperationStatus ContinuePendingRead /// /// - internal OperationStatus ContinuePendingRMW(AsyncIOContext request, + internal OperationStatus ContinuePendingRMW(AsyncIOContext request, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - ref TKey key = ref pendingContext.key.Get(); + var diskRecord = new DiskLogRecord(ref request); SpinWaitUntilClosed(request.logicalAddress); - byte* recordPointer = request.record.GetValidPointer(); - var requestRecordInfo = hlog.GetInfoFromBytePointer(recordPointer); // Not ref, as we don't want to write into request.record - ref var srcRecordInfo = ref requestRecordInfo; - srcRecordInfo.ClearBitsForDiskImages(); - OperationStatus status; while (true) { - OperationStackContext stackCtx = new(pendingContext.keyHash); - if (!FindOrCreateTagAndTryTransientXLock(sessionFunctions, ref key, ref stackCtx, out status)) + OperationStackContext stackCtx = new(pendingContext.keyHash); + if (!FindOrCreateTagAndTryEphemeralXLock(sessionFunctions, ref stackCtx, out status)) goto CheckRetry; try { // During the pending operation a record for the key may have been added to the log. If so, break and go through the full InternalRMW sequence; // the record in 'request' is stale. We only lock for tag-chain stability during search. - if (TryFindRecordForPendingOperation(ref key, ref stackCtx, hlogBase.HeadAddress, out status, ref pendingContext)) + if (TryFindRecordForPendingOperation(diskRecord.Key, ref stackCtx, out status, ref pendingContext)) { if (status != OperationStatus.SUCCESS) goto CheckRetry; @@ -219,24 +211,23 @@ internal OperationStatus ContinuePendingRMW InitialLatestLogicalAddress, then it means InitialLatestLogicalAddress is // now below HeadAddress and there is at least one record below HeadAddress but above InitialLatestLogicalAddress. We must do InternalRMW. - if (stackCtx.recSrc.LogicalAddress > pendingContext.InitialLatestLogicalAddress) + if (stackCtx.recSrc.LogicalAddress > pendingContext.initialLatestLogicalAddress) { - Debug.Assert(pendingContext.InitialLatestLogicalAddress < hlogBase.HeadAddress, "Failed to search all in-memory records"); + Debug.Assert(pendingContext.initialLatestLogicalAddress < hlogBase.HeadAddress, "Failed to search all in-memory records"); break; } // Here, the input data for 'doingCU' is the from the request, so populate the RecordSource copy from that, preserving LowestReadCache*. stackCtx.recSrc.LogicalAddress = request.logicalAddress; - stackCtx.recSrc.PhysicalAddress = (long)recordPointer; - status = CreateNewRecordRMW(ref key, ref pendingContext.input.Get(), ref hlog.GetContextRecordValue(ref request), ref pendingContext.output, - ref pendingContext, sessionFunctions, ref stackCtx, ref srcRecordInfo, - doingCU: request.logicalAddress >= hlogBase.BeginAddress && !srcRecordInfo.Tombstone); + status = CreateNewRecordRMW(diskRecord.Key, ref diskRecord, ref pendingContext.input.Get(), ref pendingContext.output, + ref pendingContext, sessionFunctions, ref stackCtx, + doingCU: request.logicalAddress >= hlogBase.BeginAddress && !diskRecord.Info.Tombstone); } finally { stackCtx.HandleNewRecordOnException(this); - TransientXUnlock(sessionFunctions, ref key, ref stackCtx); + EphemeralXUnlock(sessionFunctions, ref stackCtx); } // Must do this *after* Unlocking. @@ -248,7 +239,7 @@ internal OperationStatus ContinuePendingRMW /// /// - internal OperationStatus ContinuePendingConditionalCopyToTail(AsyncIOContext request, + internal OperationStatus ContinuePendingConditionalCopyToTail(AsyncIOContext request, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // If the key was found at or above minAddress, do nothing. // If we're here we know the key matches because AllocatorBase.AsyncGetFromDiskCallback skips colliding keys by following the .PreviousAddress chain. @@ -286,24 +277,23 @@ internal OperationStatus ContinuePendingConditionalCopyToTail stackCtx = new(storeFunctions.GetKeyHashCode64(ref key)); + var diskRecord = new DiskLogRecord(ref request); + OperationStackContext stackCtx = new(pendingContext.keyHash); // See if the record was added above the highest address we checked before issuing the IO. - var minAddress = pendingContext.InitialLatestLogicalAddress + 1; + var minAddress = pendingContext.initialLatestLogicalAddress + 1; OperationStatus internalStatus; do { - if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx, currentAddress: request.logicalAddress, minAddress, pendingContext.maxAddress, out internalStatus, out bool needIO)) + if (TryFindRecordInMainLogForConditionalOperation(sessionFunctions, diskRecord.Key, 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(sessionFunctions, ref pendingContext, ref key, ref pendingContext.input.Get(), ref pendingContext.value.Get(), - ref pendingContext.output, pendingContext.userContext, ref stackCtx, minAddress, pendingContext.maxAddress, WriteReason.Compaction) - : ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref key, ref pendingContext.input.Get(), ref pendingContext.value.Get(), - ref pendingContext.output, pendingContext.userContext, ref stackCtx, pendingContext.writeReason); + ? PrepareIOForConditionalOperation(ref pendingContext, ref diskRecord, ref stackCtx, minAddress, pendingContext.maxAddress) + : ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref diskRecord, ref stackCtx); } } while (sessionFunctions.Store.HandleImmediateNonPendingRetryStatus(internalStatus, sessionFunctions)); @@ -333,9 +323,9 @@ internal OperationStatus ContinuePendingConditionalCopyToTail /// /// - internal OperationStatus ContinuePendingConditionalScanPush(AsyncIOContext request, + internal OperationStatus ContinuePendingConditionalScanPush(AsyncIOContext request, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // If the key was found at or above minAddress, do nothing; we'll push it when we get to it. If we flagged the iteration to stop, do nothing. // If we're here we know the key matches because AllocatorBase.AsyncGetFromDiskCallback skips colliding keys by following the .PreviousAddress chain. @@ -345,8 +335,8 @@ internal OperationStatus ContinuePendingConditionalScanPush(sessionFunctions, pendingContext.scanCursorState, pendingContext.recordInfo, ref pendingContext.key.Get(), ref pendingContext.value.Get(), - currentAddress: request.logicalAddress, minAddress: pendingContext.InitialLatestLogicalAddress + 1, maxAddress: pendingContext.maxAddress); + _ = hlogBase.ConditionalScanPush>(sessionFunctions, + pendingContext.scanCursorState, ref pendingContext, currentAddress: request.logicalAddress, minAddress: pendingContext.initialLatestLogicalAddress + 1, maxAddress: pendingContext.maxAddress); // ConditionalScanPush has already called HandleOperationStatus, so return SUCCESS here. return OperationStatus.SUCCESS; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/EpochOperations.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/EpochOperations.cs index aa79edbded9..b5db8780133 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/EpochOperations.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/EpochOperations.cs @@ -7,16 +7,16 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void SynchronizeEpoch( TsavoriteExecutionContext sessionCtx, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var version = sessionCtx.version; Debug.Assert(sessionCtx.version == version, $"sessionCtx.version ({sessionCtx.version}) should == version ({version})"); @@ -38,7 +38,7 @@ void SpinWaitUntilClosed(long address) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - void SpinWaitUntilRecordIsClosed(long logicalAddress, AllocatorBase log) + void SpinWaitUntilRecordIsClosed(long logicalAddress, AllocatorBase log) { Debug.Assert(logicalAddress < log.HeadAddress, "SpinWaitUntilRecordIsClosed should not be called for addresses above HeadAddress"); 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 0c5a82c5fb8..5521c9db464 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 @@ -1,61 +1,62 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; using static Tsavorite.core.Utility; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryFindRecordInMemory(ref TKey key, ref OperationStackContext stackCtx, long minAddress, bool stopAtHeadAddress = true) + private bool TryFindRecordInMemory(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress, bool stopAtHeadAddress = true) { - if (UseReadCache && FindInReadCache(ref key, ref stackCtx, minAddress: Constants.kInvalidAddress)) + if (UseReadCache && FindInReadCache(key, ref stackCtx, minAddress: Constants.kInvalidAddress)) return true; if (minAddress < hlogBase.HeadAddress && stopAtHeadAddress) minAddress = hlogBase.HeadAddress; - return TryFindRecordInMainLog(ref key, ref stackCtx, minAddress: minAddress); + return TryFindRecordInMainLog(key, ref stackCtx, minAddress: minAddress); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryFindRecordInMemory(ref TKey key, ref OperationStackContext stackCtx, + private bool TryFindRecordInMemory(ReadOnlySpan key, ref OperationStackContext stackCtx, ref PendingContext pendingContext) { // Add 1 to the pendingContext minAddresses because we don't want an inclusive search; we're looking to see if it was added *after*. if (UseReadCache) { - var minRC = IsReadCache(pendingContext.InitialEntryAddress) ? pendingContext.InitialEntryAddress + 1 : Constants.kInvalidAddress; - if (FindInReadCache(ref key, ref stackCtx, minAddress: minRC)) + var minRC = IsReadCache(pendingContext.initialEntryAddress) ? pendingContext.initialEntryAddress + 1 : Constants.kInvalidAddress; + if (FindInReadCache(key, ref stackCtx, minAddress: minRC)) return true; } - var minLog = pendingContext.InitialLatestLogicalAddress < hlogBase.HeadAddress ? hlogBase.HeadAddress : pendingContext.InitialLatestLogicalAddress + 1; - return TryFindRecordInMainLog(ref key, ref stackCtx, minAddress: minLog); + var minLog = pendingContext.initialLatestLogicalAddress < hlogBase.HeadAddress ? hlogBase.HeadAddress : pendingContext.initialLatestLogicalAddress + 1; + return TryFindRecordInMainLog(key, ref stackCtx, minAddress: minLog); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool TryFindRecordInMainLog(ref TKey key, ref OperationStackContext stackCtx, long minAddress) + internal bool TryFindRecordInMainLog(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress) { Debug.Assert(!stackCtx.recSrc.HasInMemorySrc, "Should not have found record before this call"); if (stackCtx.recSrc.LogicalAddress >= minAddress) { stackCtx.recSrc.SetPhysicalAddress(); - TraceBackForKeyMatch(ref key, ref stackCtx.recSrc, minAddress); + TraceBackForKeyMatch(key, ref stackCtx.recSrc, minAddress); } return stackCtx.recSrc.HasInMemorySrc; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool TryFindRecordInMainLog(ref TKey key, ref OperationStackContext stackCtx, long minAddress, long maxAddress) + internal bool TryFindRecordInMainLog(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress, long maxAddress) { Debug.Assert(!stackCtx.recSrc.HasInMemorySrc, "Should not have found record before this call"); if (stackCtx.recSrc.LogicalAddress >= minAddress) { stackCtx.recSrc.SetPhysicalAddress(); - TraceBackForKeyMatch(ref key, ref stackCtx.recSrc, minAddress, maxAddress); + TraceBackForKeyMatch(key, ref stackCtx.recSrc, minAddress, maxAddress); } return stackCtx.recSrc.HasInMemorySrc; } @@ -63,8 +64,8 @@ internal bool TryFindRecordInMainLog(ref TKey key, ref OperationStackContext(TSessionFunctionsWrapper sessionFunctions, - ref TKey key, ref OperationStackContext stackCtx, long currentAddress, long minAddress, long maxAddress, out OperationStatus internalStatus, out bool needIO) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ReadOnlySpan key, ref OperationStackContext stackCtx, long currentAddress, long minAddress, long maxAddress, out OperationStatus internalStatus, out bool needIO) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { if (!FindTag(ref stackCtx.hei)) { @@ -92,8 +93,8 @@ internal bool TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx, out internalStatus)) + // The EphemeralSLock here is necessary only for the tag chain to avoid record elision/revivification during traceback. + if (!TryEphemeralSLock(sessionFunctions, ref stackCtx, out internalStatus)) return needIO = false; } else @@ -106,7 +107,7 @@ internal bool TryFindRecordInMainLogForConditionalOperation= minAddress && stackCtx.recSrc.LogicalAddress < hlogBase.HeadAddress && stackCtx.recSrc.LogicalAddress >= hlogBase.BeginAddress; @@ -114,7 +115,7 @@ internal bool TryFindRecordInMainLogForConditionalOperation(sessionFunctions, ref key, ref stackCtx); + EphemeralSUnlock(sessionFunctions, ref stackCtx); } } @@ -123,18 +124,18 @@ internal bool TryFindRecordInMainLogForConditionalOperation !recordInfo.Invalid || recordInfo.IsSealed; [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TraceBackForKeyMatch(ref TKey key, ref RecordSource recSrc, long minAddress) + private bool TraceBackForKeyMatch(ReadOnlySpan key, ref RecordSource recSrc, long minAddress) { - // PhysicalAddress must already be populated by callers. - ref var recordInfo = ref recSrc.GetInfo(); - if (IsValidTracebackRecord(recordInfo) && storeFunctions.KeysEqual(ref key, ref recSrc.GetKey())) + var logRecord = hlog.CreateLogRecord(recSrc.LogicalAddress); + + if (IsValidTracebackRecord(logRecord.Info) && storeFunctions.KeysEqual(key, logRecord.Key)) { recSrc.SetHasMainLogSrc(); return true; } - recSrc.LogicalAddress = recordInfo.PreviousAddress; - if (TraceBackForKeyMatch(ref key, recSrc.LogicalAddress, minAddress, out recSrc.LogicalAddress, out recSrc.PhysicalAddress)) + recSrc.LogicalAddress = logRecord.Info.PreviousAddress; + if (TraceBackForKeyMatch(key, recSrc.LogicalAddress, minAddress, out recSrc.LogicalAddress, out recSrc.PhysicalAddress)) { recSrc.SetHasMainLogSrc(); return true; @@ -144,18 +145,18 @@ private bool TraceBackForKeyMatch(ref TKey key, ref RecordSource recSrc, long minAddress, long maxAddress) + private bool TraceBackForKeyMatch(ReadOnlySpan key, ref RecordSource recSrc, long minAddress, long maxAddress) { - // PhysicalAddress must already be populated by callers. - ref var recordInfo = ref recSrc.GetInfo(); - if (IsValidTracebackRecord(recordInfo) && recSrc.LogicalAddress < maxAddress && storeFunctions.KeysEqual(ref key, ref recSrc.GetKey())) + var logRecord = hlog.CreateLogRecord(recSrc.LogicalAddress); + + if (IsValidTracebackRecord(logRecord.Info) && recSrc.LogicalAddress < maxAddress && storeFunctions.KeysEqual(key, logRecord.Key)) { recSrc.SetHasMainLogSrc(); return true; } - recSrc.LogicalAddress = recordInfo.PreviousAddress; - if (TraceBackForKeyMatch(ref key, recSrc.LogicalAddress, minAddress, maxAddress, out recSrc.LogicalAddress, out recSrc.PhysicalAddress)) + recSrc.LogicalAddress = logRecord.Info.PreviousAddress; + if (TraceBackForKeyMatch(key, recSrc.LogicalAddress, minAddress, maxAddress, out recSrc.LogicalAddress, out recSrc.PhysicalAddress)) { recSrc.SetHasMainLogSrc(); return true; @@ -164,19 +165,19 @@ private bool TraceBackForKeyMatch(ref TKey key, ref RecordSource key, long fromLogicalAddress, long minAddress, out long foundLogicalAddress, out long foundPhysicalAddress) { // This overload is called when the record at the "current" logical address does not match 'key'; fromLogicalAddress is its .PreviousAddress. foundLogicalAddress = fromLogicalAddress; while (foundLogicalAddress >= minAddress) { - foundPhysicalAddress = hlog.GetPhysicalAddress(foundLogicalAddress); + var logRecord = hlog.CreateLogRecord(foundLogicalAddress); + foundPhysicalAddress = logRecord.physicalAddress; - ref var recordInfo = ref hlog.GetInfo(foundPhysicalAddress); - if (IsValidTracebackRecord(recordInfo) && storeFunctions.KeysEqual(ref key, ref hlog.GetKey(foundPhysicalAddress))) + if (IsValidTracebackRecord(logRecord.Info) && storeFunctions.KeysEqual(key, logRecord.Key)) return true; - foundLogicalAddress = recordInfo.PreviousAddress; + foundLogicalAddress = logRecord.Info.PreviousAddress; } foundPhysicalAddress = Constants.kInvalidAddress; return false; @@ -184,32 +185,32 @@ private bool TraceBackForKeyMatch(ref TKey key, long fromLogicalAddress, long mi // Overload with maxAddress to avoid the extra condition - TODO: check that this duplication saves on IL/perf [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TraceBackForKeyMatch(ref TKey key, long fromLogicalAddress, long minAddress, long maxAddress, out long foundLogicalAddress, out long foundPhysicalAddress) + private bool TraceBackForKeyMatch(ReadOnlySpan key, long fromLogicalAddress, long minAddress, long maxAddress, out long foundLogicalAddress, out long foundPhysicalAddress) { // This overload is called when the record at the "current" logical address does not match 'key'; fromLogicalAddress is its .PreviousAddress. foundLogicalAddress = fromLogicalAddress; while (foundLogicalAddress >= minAddress) { - foundPhysicalAddress = hlog.GetPhysicalAddress(foundLogicalAddress); + var logRecord = hlog.CreateLogRecord(foundLogicalAddress); + foundPhysicalAddress = logRecord.physicalAddress; - ref var recordInfo = ref hlog.GetInfo(foundPhysicalAddress); - if (IsValidTracebackRecord(recordInfo) && foundLogicalAddress < maxAddress && storeFunctions.KeysEqual(ref key, ref hlog.GetKey(foundPhysicalAddress))) + if (IsValidTracebackRecord(logRecord.Info) && foundLogicalAddress < maxAddress && storeFunctions.KeysEqual(key, logRecord.Key)) return true; - foundLogicalAddress = recordInfo.PreviousAddress; + foundLogicalAddress = logRecord.Info.PreviousAddress; } foundPhysicalAddress = Constants.kInvalidAddress; return false; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryFindRecordForUpdate(ref TKey key, ref OperationStackContext stackCtx, long minAddress, out OperationStatus internalStatus) + private bool TryFindRecordForUpdate(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress, out OperationStatus internalStatus) { // This routine returns true if we should proceed with the InternalXxx operation (whether the record was found or not), // else false (including false if we need a RETRY). If it returns true with recSrc.HasInMemorySrc, caller must set srcRecordInfo. // We are not here from Read() so have not processed readcache; search that as well as the in-memory log. - if (TryFindRecordInMemory(ref key, ref stackCtx, minAddress)) + if (TryFindRecordInMemory(key, ref stackCtx, minAddress)) { if (stackCtx.recSrc.GetInfo().IsClosed) { @@ -222,13 +223,13 @@ private bool TryFindRecordForUpdate(ref TKey key, ref OperationStackContext stackCtx, long minAddress, out OperationStatus internalStatus) + private bool TryFindRecordForRead(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress, out OperationStatus internalStatus) { // This routine returns true if we should proceed with the InternalXxx operation (whether the record was found or not), // else false (including false if we need a RETRY). If it returns true with recSrc.HasInMemorySrc, caller must set srcRecordInfo. // We are here for Read() so we have already processed readcache and are just here for the traceback in the main log. - if (TryFindRecordInMainLog(ref key, ref stackCtx, minAddress)) + if (TryFindRecordInMainLog(key, ref stackCtx, minAddress)) { if (stackCtx.recSrc.GetInfo().IsClosed) { @@ -241,13 +242,13 @@ private bool TryFindRecordForRead(ref TKey key, ref OperationStackContext(ref TKey key, ref OperationStackContext stackCtx, long minAddress, out OperationStatus internalStatus, + private bool TryFindRecordForPendingOperation(ReadOnlySpan key, ref OperationStackContext stackCtx, out OperationStatus internalStatus, ref PendingContext pendingContext) { // This routine returns true if we find the key, else false. internalStatus = OperationStatus.SUCCESS; - if (!TryFindRecordInMemory(ref key, ref stackCtx, ref pendingContext)) + if (!TryFindRecordInMemory(key, ref stackCtx, ref pendingContext)) return false; if (stackCtx.recSrc.GetInfo().IsClosed) internalStatus = OperationStatus.RETRY_LATER; @@ -257,14 +258,14 @@ private bool TryFindRecordForPendingOperation(ref TKe } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryFindRecordInMainLogForPendingOperation(ref TKey key, ref OperationStackContext stackCtx, long minAddress, long maxAddress, out OperationStatus internalStatus) + private bool TryFindRecordInMainLogForPendingOperation(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress, long maxAddress, out OperationStatus internalStatus) { // This overload is called when we do not have a PendingContext to get minAddress from, and we've skipped the readcache if present. // This routine returns true if we find the key, else false. internalStatus = OperationStatus.SUCCESS; - if (!TryFindRecordInMainLog(ref key, ref stackCtx, minAddress, maxAddress)) + if (!TryFindRecordInMainLog(key, ref stackCtx, minAddress, maxAddress)) return false; if (stackCtx.recSrc.GetInfo().IsClosed) internalStatus = OperationStatus.RETRY_LATER; 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 42c0e3b462b..cc8e70834d4 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 @@ -4,20 +4,19 @@ using System.Diagnostics; using System.Runtime.CompilerServices; using System.Threading; -using System.Threading.Tasks; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool HandleImmediateRetryStatus( OperationStatus internalStatus, TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper => (internalStatus & OperationStatus.BASIC_MASK) > OperationStatus.MAX_MAP_TO_COMPLETED_STATUSCODE && HandleRetryStatus(internalStatus, sessionFunctions, ref pendingContext); @@ -26,17 +25,17 @@ private bool HandleImmediateRetryStatus [MethodImpl(MethodImplOptions.AggressiveInlining)] internal bool HandleImmediateNonPendingRetryStatus(OperationStatus internalStatus, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { Debug.Assert(epoch.ThisInstanceProtected()); switch (internalStatus) { case OperationStatus.RETRY_NOW: - Thread.Yield(); + _ = Thread.Yield(); return true; case OperationStatus.RETRY_LATER: InternalRefresh(sessionFunctions); - Thread.Yield(); + _ = Thread.Yield(); return true; default: return false; @@ -47,17 +46,17 @@ private bool HandleRetryStatus pendingContext) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { Debug.Assert(epoch.ThisInstanceProtected()); switch (internalStatus) { case OperationStatus.RETRY_NOW: - Thread.Yield(); + _ = Thread.Yield(); return true; case OperationStatus.RETRY_LATER: InternalRefresh(sessionFunctions); - Thread.Yield(); + _ = Thread.Yield(); return true; case OperationStatus.CPR_SHIFT_DETECTED: // Retry as (v+1) Operation @@ -66,8 +65,6 @@ private bool HandleRetryStatus( TsavoriteExecutionContext sessionCtx, ref PendingContext pendingContext, OperationStatus operationStatus, - out AsyncIOContext request) + out AsyncIOContext request) { Debug.Assert(operationStatus != OperationStatus.RETRY_NOW, "OperationStatus.RETRY_NOW should have been handled before HandleOperationStatus"); Debug.Assert(operationStatus != OperationStatus.RETRY_LATER, "OperationStatus.RETRY_LATER should have been handled before HandleOperationStatus"); @@ -128,8 +125,8 @@ internal Status HandleOperationStatus( if (operationStatus == OperationStatus.ALLOCATE_FAILED) { - Debug.Assert(pendingContext.IsAsync, "Sync ops should have handled ALLOCATE_FAILED before HandleOperationStatus"); Debug.Assert(!pendingContext.flushEvent.IsDefault(), "Expected flushEvent for ALLOCATE_FAILED"); + Debug.Fail("Should have handled ALLOCATE_FAILED before HandleOperationStatus"); return new(StatusCode.Pending); } else if (operationStatus == OperationStatus.RECORD_ON_DISK) @@ -141,21 +138,18 @@ internal Status HandleOperationStatus( // Issue asynchronous I/O request request.id = pendingContext.id; - request.request_key = pendingContext.key; + request.request_key = PinnedSpanByte.FromPinnedSpan(pendingContext.Key); request.logicalAddress = pendingContext.logicalAddress; request.minAddress = pendingContext.minAddress; request.record = default; - if (pendingContext.IsAsync) - request.asyncOperation = new TaskCompletionSource>(TaskCreationOptions.RunContinuationsAsynchronously); - else - request.callbackQueue = sessionCtx.readyResponses; + request.callbackQueue = sessionCtx.readyResponses; - hlogBase.AsyncGetFromDisk(pendingContext.logicalAddress, hlog.GetAverageRecordSize(), request); + hlogBase.AsyncGetFromDisk(pendingContext.logicalAddress, DiskLogRecord.InitialIOSize, request); return new(StatusCode.Pending); } else { - Debug.Assert(pendingContext.IsAsync, "Sync ops should never return status.IsFaulted"); + Debug.Fail($"Unexpected OperationStatus {operationStatus}"); return new(StatusCode.Error); } } 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 05b6dd99952..e67bd020c91 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 @@ -1,15 +1,16 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; using static Tsavorite.core.Utility; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { private enum LatchDestination { @@ -19,14 +20,21 @@ private enum LatchDestination } [MethodImpl(MethodImplOptions.AggressiveInlining)] - static ref RecordInfo WriteNewRecordInfo(ref TKey key, AllocatorBase log, long newPhysicalAddress, bool inNewVersion, long previousAddress) + static LogRecord WriteNewRecordInfo(ReadOnlySpan key, AllocatorBase log, long logicalAddress, long physicalAddress, bool inNewVersion, long previousAddress) { - ref RecordInfo recordInfo = ref log._wrapper.GetInfo(newPhysicalAddress); + ref var recordInfo = ref LogRecord.GetInfoRef(physicalAddress); recordInfo.WriteInfo(inNewVersion, previousAddress); - log._wrapper.SerializeKey(ref key, newPhysicalAddress); - return ref recordInfo; + var logRecord = log._wrapper.CreateLogRecord(logicalAddress, physicalAddress); + log._wrapper.SerializeKey(key, logicalAddress, ref logRecord); + return logRecord; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + void DisposeRecord(ref LogRecord logRecord, DisposeReason disposeReason) => hlog.DisposeRecord(ref logRecord, disposeReason); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void DisposeRecord(ref DiskLogRecord logRecord, DisposeReason disposeReason) => hlog.DisposeRecord(ref logRecord, disposeReason); + [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void MarkPage(long logicalAddress, TsavoriteExecutionContext sessionCtx) { @@ -70,7 +78,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 hlog.GetInfo(hlog.GetPhysicalAddress(entry.Address)).IsInNewVersion; + return LogRecord.GetInfo(hlog.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 @@ -78,20 +86,20 @@ private bool IsEntryVersionNew(ref HashBucketEntry entry) // Also, it cannot be elided if it is frozen due to checkpointing. [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool CanElide(TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, RecordInfo srcRecordInfo) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { Debug.Assert(!stackCtx.recSrc.HasReadCacheSrc, "Should not call CanElide() for readcache records"); return stackCtx.hei.Address == stackCtx.recSrc.LogicalAddress && srcRecordInfo.PreviousAddress < hlogBase.BeginAddress - && !IsFrozen(sessionFunctions, ref stackCtx, ref srcRecordInfo); + && !IsFrozen(sessionFunctions, ref stackCtx, srcRecordInfo); } // If the record is in a checkpoint range, it must not be modified. If it is in the fuzzy region, it can only be modified // if it is a new record. [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool IsFrozen(TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, RecordInfo srcRecordInfo) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { Debug.Assert(!stackCtx.recSrc.HasReadCacheSrc, "Should not call IsFrozen() for readcache records"); return sessionFunctions.Ctx.IsInV1 @@ -99,42 +107,63 @@ private bool IsFrozen(TSess || !srcRecordInfo.IsInNewVersion); // In fuzzy region and an old version } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal long GetMinRevivifiableAddress() + => RevivificationManager.GetMinRevivifiableAddress(hlogBase.GetTailAddress(), hlogBase.ReadOnlyAddress); + [MethodImpl(MethodImplOptions.AggressiveInlining)] private (bool elided, bool added) TryElideAndTransferToFreeList(TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, (int usedValueLength, int fullValueLength, int fullRecordLength) recordLengths) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, ref LogRecord logRecord) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // Try to CAS out of the hashtable and if successful, add it to the free list. - Debug.Assert(srcRecordInfo.IsSealed, "Expected a Sealed record in TryElideAndTransferToFreeList"); + Debug.Assert(logRecord.Info.IsSealed, "Expected a Sealed record in TryElideAndTransferToFreeList"); if (!stackCtx.hei.TryElide()) return (false, false); - return (true, TryTransferToFreeList(sessionFunctions, ref stackCtx, ref srcRecordInfo, recordLengths)); + return (true, TryTransferToFreeList(sessionFunctions, ref stackCtx, ref logRecord)); } [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool TryTransferToFreeList(TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, - ref RecordInfo srcRecordInfo, (int usedValueLength, int fullValueLength, int fullRecordLength) recordLengths) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, 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(srcRecordInfo.IsSealed, "Expected a Sealed record in TryTransferToFreeList"); + Debug.Assert(logRecord.Info.IsSealed, "Expected a Sealed record in TryTransferToFreeList"); // 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. - // We don't want the caller to know details of the Filler, so we cleared out any extraValueLength entry to ensure the space beyond - // usedValueLength is zero'd for log-scan correctness. - ref TValue recordValue = ref stackCtx.recSrc.GetValue(); - ClearExtraValueSpace(ref srcRecordInfo, ref recordValue, recordLengths.usedValueLength, recordLengths.fullValueLength); - storeFunctions.DisposeRecord(ref stackCtx.recSrc.GetKey(), ref recordValue, DisposeReason.RevivificationFreeList); + 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; - SetFreeRecordSize(stackCtx.recSrc.PhysicalAddress, ref srcRecordInfo, recordLengths.fullRecordLength); - return RevivificationManager.TryAdd(stackCtx.recSrc.LogicalAddress, recordLengths.fullRecordLength, ref sessionFunctions.Ctx.RevivificationStats); + return RevivificationManager.TryAdd(stackCtx.recSrc.LogicalAddress, ref logRecord, ref sessionFunctions.Ctx.RevivificationStats); + } + + // Do not try to inline this; it causes TryAllocateRecord to bloat and slow + bool TryTakeFreeRecord(TSessionFunctionsWrapper sessionFunctions, ref RecordSizeInfo sizeInfo, long minRevivAddress, + out long logicalAddress, out long physicalAddress, out int allocatedSize) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + { + // Caller checks for UseFreeRecordPool + if (RevivificationManager.TryTake(ref sizeInfo, minRevivAddress, out logicalAddress, ref sessionFunctions.Ctx.RevivificationStats)) + { + var logRecord = hlog.CreateLogRecord(logicalAddress); + Debug.Assert(logRecord.Info.IsSealed, "TryTakeFreeRecord: recordInfo should still have the revivification Seal"); + + // Preserve the Sealed bit due to checkpoint/recovery; see RecordInfo.WriteInfo. + physicalAddress = logRecord.physicalAddress; + allocatedSize = logRecord.GetInlineRecordSizes().allocatedSize; + return true; + } + + // No free record available. + logicalAddress = physicalAddress = default; + allocatedSize = default; + return false; } internal enum LatchOperation : byte @@ -148,50 +177,52 @@ internal void SetRecordInvalid(long logicalAddress) { // This is called on exception recovery for a newly-inserted record. var localLog = IsReadCache(logicalAddress) ? readcache : hlog; - ref var recordInfo = ref localLog.GetInfo(localLog.GetPhysicalAddress(AbsoluteAddress(logicalAddress))); - recordInfo.SetInvalid(); + LogRecord.GetInfoRef(localLog.GetPhysicalAddress(AbsoluteAddress(logicalAddress))).SetInvalid(); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool CASRecordIntoChain(ref TKey key, ref OperationStackContext stackCtx, long newLogicalAddress, ref RecordInfo newRecordInfo) + private bool CASRecordIntoChain(long newLogicalAddress, ref LogRecord newLogRecord, ref OperationStackContext stackCtx) { var result = stackCtx.recSrc.LowestReadCachePhysicalAddress == Constants.kInvalidAddress ? stackCtx.hei.TryCAS(newLogicalAddress) - : SpliceIntoHashChainAtReadCacheBoundary(ref key, ref stackCtx, newLogicalAddress); + : SpliceIntoHashChainAtReadCacheBoundary(newLogRecord.Key, ref stackCtx, newLogicalAddress); if (result) - newRecordInfo.UnsealAndValidate(); + newLogRecord.InfoRef.UnsealAndValidate(); return result; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void PostCopyToTail(ref TKey key, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - => PostCopyToTail(ref key, ref stackCtx, ref srcRecordInfo, stackCtx.hei.Address); + private void PostCopyToTail(ref TSourceLogRecord srcLogRecord, ref OperationStackContext stackCtx) + where TSourceLogRecord : ISourceLogRecord + => PostCopyToTail(ref srcLogRecord, ref stackCtx, stackCtx.hei.Address); [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void PostCopyToTail(ref TKey key, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, long highestReadCacheAddressChecked) + private void PostCopyToTail(ref TSourceLogRecord srcLogRecord, ref OperationStackContext stackCtx, long highestReadCacheAddressChecked) + where TSourceLogRecord : ISourceLogRecord { // Nothing required here if not using ReadCache if (!UseReadCache) return; + // We're using the read cache, so any insertion must check that a readcache insertion wasn't done if (stackCtx.recSrc.HasReadCacheSrc) { // If we already have a readcache source, there will not be another inserted, so we can just invalidate the source directly. - srcRecordInfo.SetInvalidAtomic(); + srcLogRecord.InfoRef.SetInvalidAtomic(); } - else + else if (stackCtx.recSrc.HasMainLogSrc) { // We did not have a readcache source, so while we spliced a new record into the readcache/mainlog gap a competing readcache record may have been inserted at the tail. // If so, invalidate it. highestReadCacheAddressChecked is hei.Address unless we are from ConditionalCopyToTail, which may have skipped the readcache before this. // See "Consistency Notes" in TryCopyToReadCache for a discussion of why there ie no "momentary inconsistency" possible here. - ReadCacheCheckTailAfterSplice(ref key, ref stackCtx.hei, highestReadCacheAddressChecked); + ReadCacheCheckTailAfterSplice(srcLogRecord.Key, ref stackCtx.hei, highestReadCacheAddressChecked); } } // Called after BlockAllocate or anything else that could shift HeadAddress, to adjust addresses or return false for RETRY as needed. // This refreshes the HashEntryInfo, so the caller needs to recheck to confirm the BlockAllocated address is still > hei.Address. [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool VerifyInMemoryAddresses(ref OperationStackContext stackCtx) + private bool VerifyInMemoryAddresses(ref OperationStackContext stackCtx) { // If we have an in-memory source that fell below HeadAddress, return false and the caller will RETRY_LATER. if (stackCtx.recSrc.HasInMemorySrc && stackCtx.recSrc.LogicalAddress < stackCtx.recSrc.AllocatorBase.HeadAddress) @@ -207,13 +238,14 @@ private bool VerifyInMemoryAddresses(ref OperationStackContext(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx, out OperationStatus internalStatus) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private bool FindOrCreateTagAndTryEphemeralXLock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx, out OperationStatus internalStatus) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - // Transient must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. Manual locking already automatically locks the bucket. + // Ephemeral must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. + // Manual locking already automatically locks the bucket. hei already has the key's hashcode. FindOrCreateTag(ref stackCtx.hei, hlogBase.BeginAddress); - if (!TryTransientXLock(sessionFunctions, ref key, ref stackCtx, out internalStatus)) + if (!TryEphemeralXLock(sessionFunctions, ref stackCtx, out internalStatus)) return false; // Between the time we found the tag and the time we locked the bucket the record in hei.entry may have been elided, so make sure we don't have a stale address in hei.entry. @@ -223,13 +255,14 @@ private bool FindOrCreateTagAndTryTransientXLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx, out OperationStatus internalStatus) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private bool FindTagAndTryEphemeralXLock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx, out OperationStatus internalStatus) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - // Transient must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. Manual locking already automatically locks the bucket. + // Ephemeral must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. + // Manual locking already automatically locks the bucket. hei already has the key's hashcode. internalStatus = OperationStatus.NOTFOUND; - if (!FindTag(ref stackCtx.hei) || !TryTransientXLock(sessionFunctions, ref key, ref stackCtx, out internalStatus)) + if (!FindTag(ref stackCtx.hei) || !TryEphemeralXLock(sessionFunctions, ref stackCtx, out internalStatus)) return false; // Between the time we found the tag and the time we locked the bucket the record in hei.entry may have been elided, so make sure we don't have a stale address in hei.entry. @@ -239,13 +272,14 @@ private bool FindTagAndTryTransientXLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx, out OperationStatus internalStatus) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private bool FindTagAndTryEphemeralSLock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx, out OperationStatus internalStatus) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - // Transient must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. Manual locking already automatically locks the bucket. + // Ephemeral must lock the bucket before traceback, to prevent revivification from yanking the record out from underneath us. + // Manual locking already automatically locks the bucket. hei already has the key's hashcode. internalStatus = OperationStatus.NOTFOUND; - if (!FindTag(ref stackCtx.hei) || !TryTransientSLock(sessionFunctions, ref key, ref stackCtx, out internalStatus)) + if (!FindTag(ref stackCtx.hei) || !TryEphemeralSLock(sessionFunctions, ref stackCtx, out internalStatus)) return false; // Between the time we found the tag and the time we locked the bucket the record in hei.entry may have been elided, so make sure we don't have a stale address in hei.entry. @@ -259,17 +293,22 @@ private bool FindTagAndTryTransientSLock( - TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, int usedValueLength, int fullValueLength, int fullRecordLength) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref LogRecord srcLogRecord) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { if (!RevivificationManager.IsEnabled) { // We are not doing revivification, so we just want to remove the record from the tag chain so we don't potentially do an IO later for key // traceback. If we succeed, we need to SealAndInvalidate. It's fine if we don't succeed here; this is just tidying up the HashBucket. if (stackCtx.hei.TryElide()) - srcRecordInfo.SealAndInvalidate(); + { + srcLogRecord.InfoRef.SealAndInvalidate(); + DisposeRecord(ref srcLogRecord, DisposeReason.Elided); + } + return; } - else if (RevivificationManager.UseFreeRecordPool) + + if (RevivificationManager.UseFreeRecordPool) { // For non-FreeRecordPool revivification, we leave the record in as a normal tombstone so we can revivify it in the chain for the same key. // For FreeRecord Pool we must first Seal here, even if we're using the LockTable, because the Sealed state must survive this Delete() call. @@ -277,19 +316,16 @@ private void HandleRecordElision(sessionFunctions, ref stackCtx, ref srcRecordInfo, - (usedValueLength, fullValueLength, fullRecordLength)); + Debug.Assert(stackCtx.recSrc.LogicalAddress < hlogBase.ReadOnlyAddress || srcLogRecord.Info.Tombstone, $"Unexpected loss of Tombstone; Record should have been XLocked or SealInvalidated. RecordInfo: {srcLogRecord.Info.ToString()}"); + var (isElided, isAdded) = TryElideAndTransferToFreeList(sessionFunctions, ref stackCtx, ref srcLogRecord); if (!isElided) { // Leave this in the chain as a normal Tombstone; we aren't going to add a new record so we can't leave this one sealed. - srcRecordInfo.UnsealAndValidate(); + srcLogRecord.InfoRef.UnsealAndValidate(); + DisposeRecord(ref srcLogRecord, DisposeReason.Deleted); } else if (!isAdded && RevivificationManager.restoreDeletedRecordsIfBinIsFull) { @@ -297,12 +333,15 @@ private void HandleRecordElision : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Delete operation. Replaces the value corresponding to 'key' with tombstone. @@ -40,29 +41,28 @@ public unsafe partial class TsavoriteKV /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalDelete(ref TKey key, long keyHash, ref TContext userContext, + internal OperationStatus InternalDelete(ReadOnlySpan key, long keyHash, ref TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var latchOperation = LatchOperation.None; - OperationStackContext stackCtx = new(keyHash); + OperationStackContext stackCtx = new(keyHash); pendingContext.keyHash = keyHash; if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); - if (!FindTagAndTryTransientXLock(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (!FindTagAndTryEphemeralXLock(sessionFunctions, ref stackCtx, out OperationStatus status)) return status; - RecordInfo dummyRecordInfo = RecordInfo.InitialValid; - ref RecordInfo srcRecordInfo = ref dummyRecordInfo; + LogRecord srcLogRecord = default; // We must use try/finally to ensure unlocking even in the presence of exceptions. try { // Search the entire in-memory region; this lets us find a tombstoned record in the immutable region, avoiding unnecessarily adding one. - if (!TryFindRecordForUpdate(ref key, ref stackCtx, hlogBase.HeadAddress, out status)) + if (!TryFindRecordForUpdate(key, ref stackCtx, hlogBase.HeadAddress, out status)) return status; // Note: Delete does not track pendingContext.InitialAddress because we don't have an InternalContinuePendingDelete @@ -78,7 +78,7 @@ internal OperationStatus InternalDelete= hlogBase.ReadOnlyAddress) { - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); // If we already have a deleted record, there's nothing to do. - if (srcRecordInfo.Tombstone) + if (srcLogRecord.Info.Tombstone) return OperationStatus.NOTFOUND; // Mutable Region: Update the record in-place - deleteInfo.SetRecordInfo(ref srcRecordInfo); - ref TValue recordValue = ref stackCtx.recSrc.GetValue(); - // DeleteInfo's lengths are filled in and GetRecordLengths and SetDeletedValueLength are called inside ConcurrentDeleter. - if (sessionFunctions.ConcurrentDeleter(stackCtx.recSrc.PhysicalAddress, ref stackCtx.recSrc.GetKey(), ref recordValue, ref deleteInfo, ref srcRecordInfo, out int fullRecordLength)) + // DeleteInfo's lengths are filled in and GetRecordLengths and SetDeletedValueLength are called inside InPlaceDeleter. + if (sessionFunctions.InPlaceDeleter(ref srcLogRecord, ref deleteInfo)) { MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); // Try to transfer the record from the tag chain to the free record pool iff previous address points to invalid address. // Otherwise an earlier record for this key could be reachable again. - if (CanElide(sessionFunctions, ref stackCtx, ref srcRecordInfo)) - { - HandleRecordElision( - sessionFunctions, ref stackCtx, ref srcRecordInfo, deleteInfo.UsedValueLength, deleteInfo.FullValueLength, fullRecordLength); - } + if (CanElide(sessionFunctions, ref stackCtx, srcLogRecord.Info)) + HandleRecordElision(sessionFunctions, ref stackCtx, ref srcLogRecord); + else + DisposeRecord(ref srcLogRecord, DisposeReason.Deleted); status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); goto LatchRelease; } + if (deleteInfo.Action == DeleteAction.CancelOperation) { status = OperationStatus.CANCELED; @@ -140,30 +138,27 @@ internal OperationStatus InternalDelete(sessionFunctions, ref key, ref stackCtx); + EphemeralXUnlock(sessionFunctions, ref stackCtx); } LatchRelease: @@ -183,19 +178,7 @@ internal OperationStatus InternalDelete(ref TKey key, TContext userContext, - ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - { - pendingContext.type = OperationType.DELETE; - if (pendingContext.key == default) pendingContext.key = hlog.GetKeyContainer(ref key); - pendingContext.userContext = userContext; - pendingContext.InitialLatestLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; - pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; - } - - private LatchDestination CheckCPRConsistencyDelete(Phase phase, ref OperationStackContext stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) + private LatchDestination CheckCPRConsistencyDelete(Phase phase, ref OperationStackContext stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) { // This is the same logic as Upsert; neither goes pending. return CheckCPRConsistencyUpsert(phase, ref stackCtx, ref status, ref latchOperation); @@ -205,26 +188,29 @@ private LatchDestination CheckCPRConsistencyDelete(Phase phase, ref OperationSta /// Create a new tombstoned record for Delete /// /// The record Key + /// The source record, if . and + /// it is either too small or is in readonly region, or is in raadcache /// Information about the operation context /// The current session - /// Contains the and structures for this operation, + /// Contains the and structures for this operation, /// and allows passing back the newLogicalAddress for invalidation in the case of exceptions. - /// If ., - /// this is the for - private OperationStatus CreateNewRecordDelete(ref TKey key, ref PendingContext pendingContext, - TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private OperationStatus CreateNewRecordDelete(ReadOnlySpan key, ref LogRecord srcLogRecord, ref PendingContext pendingContext, + TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - var value = default(TValue); - var (actualSize, allocatedSize, keySize) = hlog.GetRecordSize(ref key, ref value); + var sizeInfo = hlog.GetDeleteRecordSize(key); + AllocateOptions allocOptions = new() + { + recycle = true, + elideSourceRecord = stackCtx.recSrc.HasMainLogSrc && CanElide(sessionFunctions, ref stackCtx, srcLogRecord.Info) + }; // We know the existing record cannot be elided; it must point to a valid record; otherwise InternalDelete would have returned NOTFOUND. - if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, actualSize, ref allocatedSize, keySize, new AllocateOptions() { Recycle = true }, - out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status)) + if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, ref sizeInfo, allocOptions, out var newLogicalAddress, out var newPhysicalAddress, out var allocatedSize, out var status)) return status; - ref RecordInfo newRecordInfo = ref WriteNewRecordInfo(ref key, hlogBase, newPhysicalAddress, inNewVersion: sessionFunctions.Ctx.InNewVersion, stackCtx.recSrc.LatestLogicalAddress); - newRecordInfo.SetTombstone(); + var newLogRecord = WriteNewRecordInfo(key, hlogBase, newLogicalAddress, newPhysicalAddress, sessionFunctions.Ctx.InNewVersion, previousAddress: stackCtx.recSrc.LatestLogicalAddress); + newLogRecord.InfoRef.SetTombstone(); stackCtx.SetNewRecord(newLogicalAddress); DeleteInfo deleteInfo = new() @@ -234,53 +220,50 @@ private OperationStatus CreateNewRecordDelete : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] internal bool InternalTryLockShared(long keyHash) 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 72c47819d7a..a5cad1cd863 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 @@ -1,14 +1,15 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Read-Modify-Write Operation. Updates value of 'key' using 'input' and current value. @@ -47,39 +48,38 @@ public unsafe partial class TsavoriteKV /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalRMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, ref TContext userContext, + internal OperationStatus InternalRMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, ref TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var latchOperation = LatchOperation.None; - OperationStackContext stackCtx = new(keyHash); + OperationStackContext stackCtx = new(keyHash); pendingContext.keyHash = keyHash; if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); - if (!FindOrCreateTagAndTryTransientXLock(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (!FindOrCreateTagAndTryEphemeralXLock(sessionFunctions, ref stackCtx, out var status)) return status; - RecordInfo dummyRecordInfo = RecordInfo.InitialValid; - ref RecordInfo srcRecordInfo = ref dummyRecordInfo; + LogRecord srcLogRecord = default; // We must use try/finally to ensure unlocking even in the presence of exceptions. try { // Search the entire in-memory region. - if (!TryFindRecordForUpdate(ref key, ref stackCtx, hlogBase.HeadAddress, out status)) + if (!TryFindRecordForUpdate(key, ref stackCtx, hlogBase.HeadAddress, out status)) return status; // These track the latest main-log address in the tag chain; InternalContinuePendingRMW uses them to check for new inserts. - pendingContext.InitialEntryAddress = stackCtx.hei.Address; - pendingContext.InitialLatestLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; + pendingContext.initialEntryAddress = stackCtx.hei.Address; + pendingContext.initialLatestLogicalAddress = stackCtx.recSrc.LatestLogicalAddress; // If there is a readcache record, use it as the CopyUpdater source. if (stackCtx.recSrc.HasReadCacheSrc) { - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); goto CreateNewRecord; } @@ -93,7 +93,7 @@ internal OperationStatus InternalRMW= hlogBase.ReadOnlyAddress) { - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); // Mutable Region: Update the record in-place. We perform mutable updates only if we are in normal processing phase of checkpointing RMWInfo rmwInfo = new() @@ -113,58 +113,52 @@ internal OperationStatus InternalRMW= GetMinRevivifiableAddress()) { - if (!sessionFunctions.NeedInitialUpdate(ref key, ref input, ref output, ref rmwInfo)) + if (!sessionFunctions.NeedInitialUpdate(key, ref input, ref output, ref rmwInfo)) { status = OperationStatus.NOTFOUND; goto LatchRelease; } - if (TryRevivifyInChain(ref key, ref input, ref output, ref pendingContext, sessionFunctions, ref stackCtx, ref srcRecordInfo, ref rmwInfo, out status, ref recordValue) + if (TryRevivifyInChain(ref srcLogRecord, ref input, ref output, ref pendingContext, sessionFunctions, ref stackCtx, ref rmwInfo, out status) || status != OperationStatus.SUCCESS) goto LatchRelease; } goto CreateNewRecord; } - // rmwInfo's lengths are filled in and GetValueLengths and SetLength are called inside InPlaceUpdater. - if (sessionFunctions.InPlaceUpdater(stackCtx.recSrc.PhysicalAddress, ref key, ref input, ref recordValue, ref output, ref rmwInfo, out status, ref srcRecordInfo)) + var sizeInfo = hlog.GetRMWCopyRecordSize(ref srcLogRecord, ref input, sessionFunctions); + if (sessionFunctions.InPlaceUpdater(ref srcLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo, out status)) { MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); - pendingContext.recordInfo = srcRecordInfo; - pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; // status has been set by InPlaceUpdater + pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; goto LatchRelease; } if (rmwInfo.Action == RMWAction.ExpireAndStop) { MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); - srcRecordInfo.SetDirtyAndModified(); + // Tombstone has been set by SessionFunctionsWrapper.InPlaceUpdater + srcLogRecord.InfoRef.SetDirtyAndModified(); // ExpireAndStop means to override default Delete handling (which is to go to InitialUpdater) by leaving the tombstoned record as current. // Our SessionFunctionsWrapper.InPlaceUpdater implementation has already reinitialized-in-place or set Tombstone as appropriate and marked the record. // Try to transfer the record from the tag chain to the free record pool iff previous address points to invalid address. // Otherwise an earlier record for this key could be reachable again. - if (CanElide(sessionFunctions, ref stackCtx, ref srcRecordInfo)) - { - HandleRecordElision( - sessionFunctions, ref stackCtx, ref srcRecordInfo, rmwInfo.UsedValueLength, rmwInfo.FullValueLength, rmwInfo.FullRecordLength); - } + if (CanElide(sessionFunctions, ref stackCtx, srcLogRecord.Info)) + HandleRecordElision(sessionFunctions, ref stackCtx, ref srcLogRecord); + else + DisposeRecord(ref srcLogRecord, DisposeReason.Deleted); - pendingContext.recordInfo = srcRecordInfo; pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; goto LatchRelease; } @@ -184,7 +178,7 @@ internal OperationStatus InternalRMW= hlogBase.BeginAddress) @@ -194,33 +188,27 @@ internal OperationStatus InternalRMW(sessionFunctions, ref key, ref stackCtx); + EphemeralXUnlock(sessionFunctions, ref stackCtx); } LatchRelease: @@ -242,72 +230,47 @@ internal OperationStatus InternalRMW(ref TKey key, ref TInput input, TOutput output, TContext userContext, - ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private void CreatePendingRMWContext(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext, + ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { pendingContext.type = OperationType.RMW; - if (pendingContext.key == default) - pendingContext.key = hlog.GetKeyContainer(ref key); - if (pendingContext.input == default) - pendingContext.input = sessionFunctions.GetHeapContainer(ref input); - - pendingContext.output = output; - sessionFunctions.ConvertOutputToHeap(ref input, ref pendingContext.output); - - pendingContext.userContext = userContext; + pendingContext.SerializeForReadOrRMW(key, ref input, ref output, userContext, sessionFunctions, hlogBase.bufferPool); pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; } - private bool TryRevivifyInChain(ref TKey key, ref TInput input, ref TOutput output, ref PendingContext pendingContext, - TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, ref RMWInfo rmwInfo, - out OperationStatus status, ref TValue recordValue) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private bool TryRevivifyInChain(ref LogRecord logRecord, ref TInput input, ref TOutput output, ref PendingContext pendingContext, + TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref RMWInfo rmwInfo, out OperationStatus status) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - if (IsFrozen(sessionFunctions, ref stackCtx, ref srcRecordInfo)) + if (IsFrozen(sessionFunctions, ref stackCtx, logRecord.Info)) goto NeedNewRecord; // This record is safe to revivify even if its PreviousAddress points to a valid record, because it is revivified for the same key. - var ok = true; + var ok = false; try { - if (srcRecordInfo.Tombstone) + var sizeInfo = hlog.GetRMWInitialRecordSize(logRecord.Key, ref input, sessionFunctions); + if (logRecord.TrySetValueLength(ref sizeInfo)) { - srcRecordInfo.ClearTombstone(); - - if (RevivificationManager.IsFixedLength) - rmwInfo.UsedValueLength = rmwInfo.FullValueLength = RevivificationManager.FixedValueLength; - else - { - var recordLengths = GetRecordLengths(stackCtx.recSrc.PhysicalAddress, ref recordValue, ref srcRecordInfo); - rmwInfo.FullValueLength = recordLengths.fullValueLength; - - // RMW uses GetInitialRecordSize because it has only the initial Input, not a Value - var (requiredSize, _, _) = hlog.GetRMWInitialRecordSize(ref key, ref input, sessionFunctions); - (ok, rmwInfo.UsedValueLength) = TryReinitializeTombstonedValue(sessionFunctions, - ref srcRecordInfo, ref key, ref recordValue, requiredSize, recordLengths); - } - - if (ok && sessionFunctions.InitialUpdater(ref key, ref input, ref recordValue, ref output, ref rmwInfo, ref srcRecordInfo)) + logRecord.InfoRef.ClearTombstone(); + logRecord.ClearOptionals(); + if (sessionFunctions.InitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) { // Success MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); - pendingContext.recordInfo = srcRecordInfo; pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); + ok = true; return true; } - - // Did not revivify; restore the tombstone and leave the deleted record there. - srcRecordInfo.SetTombstone(); } + // Did not revivify; restore the tombstone in 'finally' and leave the deleted record there. } finally { - if (ok) - SetExtraValueLength(ref recordValue, ref srcRecordInfo, rmwInfo.UsedValueLength, rmwInfo.FullValueLength); - else - SetTombstoneAndExtraValueLength(ref recordValue, ref srcRecordInfo, rmwInfo.UsedValueLength, rmwInfo.FullValueLength); // Restore tombstone and ensure default value on inability to update in place + if (!ok) + logRecord.InfoRef.SetTombstone(); } NeedNewRecord: @@ -316,7 +279,7 @@ private bool TryRevivifyInChain stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) + private LatchDestination CheckCPRConsistencyRMW(Phase phase, ref OperationStackContext stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) { // The idea of CPR is that if a thread in version V tries to perform an operation and notices a record in V+1, it needs to back off and run CPR_SHIFT_DETECTED. // Similarly, a V+1 thread cannot update a V record; it needs to do a read-copy-update (or upsert at tail) instead of an in-place update. @@ -355,31 +318,26 @@ private LatchDestination CheckCPRConsistencyRMW(Phase phase, ref OperationStackC /// /// Create a new record for RMW /// - /// - /// - /// - /// - /// The record Key - /// Input to the operation - /// Old value - /// The result of ISessionFunctions.SingleWriter + /// Key, if inserting a new record. + /// The source record. If . + /// it is in-memory (either too small or is in readonly region, or is in readcache); otherwise it is from disk IO + /// Input to the ISessionFunctions operation + /// The result of ISessionFunctions operation /// Information about the operation context /// The current session - /// Contains the and structures for this operation, + /// Contains the and structures for this operation, /// and allows passing back the newLogicalAddress for invalidation in the case of exceptions. If called from pending IO, /// this is populated from the data read from disk. - /// If ., - /// this is the for . Otherwise, if called from pending IO, - /// this is the read from disk. If neither of these, it is a default . - /// Whether we are doing a CopyUpdate, either from in-memory or pending IO + /// Whether we are doing a CopyUpdate, either from in-memory or pending IO. /// - private OperationStatus CreateNewRecordRMW(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, + private OperationStatus CreateNewRecordRMW(ReadOnlySpan key, ref TSourceLogRecord srcLogRecord, ref TInput input, ref TOutput output, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, bool doingCU) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, bool doingCU) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - bool forExpiration = false; - bool addTombstone = false; + var forExpiration = false; + var addTombstone = false; RetryNow: @@ -389,20 +347,18 @@ private OperationStatus CreateNewRecordRMW(sessionFunctions, ref stackCtx, ref srcRecordInfo) + recycle = true, + elideSourceRecord = stackCtx.recSrc.HasMainLogSrc && CanElide(sessionFunctions, ref stackCtx, srcLogRecord.Info) }; // Perform Need* if (doingCU) { - rmwInfo.SetRecordInfo(ref srcRecordInfo); - if (!sessionFunctions.NeedCopyUpdate(ref key, ref input, ref value, ref output, ref rmwInfo)) + if (!sessionFunctions.NeedCopyUpdate(ref srcLogRecord, ref input, ref output, ref rmwInfo)) { if (rmwInfo.Action == RMWAction.CancelOperation) return OperationStatus.CANCELED; @@ -413,14 +369,14 @@ private OperationStatus CreateNewRecordRMW( - sessionFunctions, ref stackCtx, ref srcRecordInfo, oldRecordLengths.usedValueLength, oldRecordLengths.fullValueLength, oldRecordLengths.fullRecordLength); + _ = srcLogRecord.AsLogRecord(out var inMemoryLogRecord); + HandleRecordElision(sessionFunctions, ref stackCtx, ref inMemoryLogRecord); // no new record created and hash entry is empty now return OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.Found | StatusCode.Expired); } @@ -434,47 +390,42 @@ private OperationStatus CreateNewRecordRMW storeFunctions.DisposeValueObject(obj, DisposeReason.CopyUpdated)); goto DoCAS; } if (rmwInfo.Action == RMWAction.CancelOperation) { // Save allocation for revivification (not retry, because this is canceling of the current operation), or abandon it if that fails. - if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, newPhysicalAddress, allocatedSize, ref sessionFunctions.Ctx.RevivificationStats)) + if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, ref newLogRecord, ref sessionFunctions.Ctx.RevivificationStats)) stackCtx.ClearNewRecord(); else - stackCtx.SetNewRecordInvalid(ref newRecordInfo); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); return OperationStatus.CANCELED; } if (rmwInfo.Action == RMWAction.ExpireAndStop) @@ -518,8 +468,8 @@ private OperationStatus CreateNewRecordRMW(ref key, ref input, ref newRecordValue, ref output, ref newRecordInfo, - ref rmwInfo, newLogicalAddress, sessionFunctions, isIpu: false, out status)) + if (!ReinitializeExpiredRecord(ref newLogRecord, ref input, ref output, ref rmwInfo, newLogicalAddress, sessionFunctions, isIpu: false, out status)) { // An IPU was not (or could not) be done. Cancel if requested, else invalidate the allocated record and retry. if (status == OperationStatus.CANCELED) return status; // Save allocation for revivification (not retry, because this may have been false because the record was too small), or abandon it if that fails. - if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, newPhysicalAddress, allocatedSize, ref sessionFunctions.Ctx.RevivificationStats)) + if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, ref newLogRecord, ref sessionFunctions.Ctx.RevivificationStats)) stackCtx.ClearNewRecord(); else - stackCtx.SetNewRecordInvalid(ref newRecordInfo); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); goto RetryNow; } goto DoCAS; @@ -550,8 +499,8 @@ private OperationStatus CreateNewRecordRMW= GetMinRevivifiableAddress()) - { - // We need to re-get the old record's length because rmwInfo has the new record's info. If freelist-add fails, it remains Sealed/Invalidated. - var oldRecordLengths = GetRecordLengths(stackCtx.recSrc.PhysicalAddress, ref hlog.GetValue(stackCtx.recSrc.PhysicalAddress), ref srcRecordInfo); - _ = TryTransferToFreeList(sessionFunctions, ref stackCtx, ref srcRecordInfo, oldRecordLengths); - } + _ = TryTransferToFreeList(sessionFunctions, ref stackCtx, ref inMemoryLogRecord); + else + DisposeRecord(ref inMemoryLogRecord, DisposeReason.Elided); } - else - srcRecordInfo.Seal(); // The record was not elided, so do not Invalidate + else if (stackCtx.recSrc.HasMainLogSrc) + srcLogRecord.InfoRef.Seal(); // The record was not elided, so do not Invalidate } stackCtx.ClearNewRecord(); - pendingContext.recordInfo = newRecordInfo; pendingContext.logicalAddress = newLogicalAddress; return status; } // CAS failed - stackCtx.SetNewRecordInvalid(ref newRecordInfo); - ref TValue insertedValue = ref hlog.GetValue(newPhysicalAddress); - ref TKey insertedKey = ref hlog.GetKey(newPhysicalAddress); - storeFunctions.DisposeRecord(ref insertedKey, ref insertedValue, doingCU ? DisposeReason.CopyUpdaterCASFailed : DisposeReason.InitialUpdaterCASFailed); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); + DisposeRecord(ref newLogRecord, doingCU ? DisposeReason.CopyUpdaterCASFailed : DisposeReason.InitialUpdaterCASFailed); - SetExtraValueLength(ref newRecordValue, ref newRecordInfo, rmwInfo.UsedValueLength, rmwInfo.FullValueLength); - SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress, allocatedSize); + SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress); return OperationStatus.RETRY_NOW; // CAS failure does not require epoch refresh } - internal bool ReinitializeExpiredRecord(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref RecordInfo recordInfo, ref RMWInfo rmwInfo, + internal bool ReinitializeExpiredRecord(ref LogRecord logRecord, ref TInput input, ref TOutput output, ref RMWInfo rmwInfo, long logicalAddress, TSessionFunctionsWrapper sessionFunctions, bool isIpu, out OperationStatus status) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // This is called for InPlaceUpdater or CopyUpdater only; CopyUpdater however does not copy an expired record, so we return CreatedRecord. var advancedStatusCode = isIpu ? StatusCode.InPlaceUpdatedRecord : StatusCode.CreatedRecord; advancedStatusCode |= StatusCode.Expired; - if (!sessionFunctions.NeedInitialUpdate(ref key, ref input, ref output, ref rmwInfo)) + if (!sessionFunctions.NeedInitialUpdate(logRecord.Key, ref input, ref output, ref rmwInfo)) { if (rmwInfo.Action == RMWAction.CancelOperation) { @@ -638,22 +583,23 @@ internal bool ReinitializeExpiredRecord= requiredSize) + logRecord.ClearOptionals(); + if (logRecord.TrySetValueLength(ref sizeInfo)) { - if (sessionFunctions.InitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo)) + if (sessionFunctions.InitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) { // If IPU path, we need to complete PostInitialUpdater as well if (isIpu) - sessionFunctions.PostInitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + sessionFunctions.PostInitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); status = OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, advancedStatusCode); return true; @@ -668,7 +614,7 @@ internal bool ReinitializeExpiredRecord : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Read operation. Computes the 'output' from 'input' and current value corresponding to 'key'. @@ -50,24 +51,20 @@ public unsafe partial class TsavoriteKV /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalRead(ref TKey key, long keyHash, ref TInput input, ref TOutput output, + internal OperationStatus InternalRead(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - OperationStackContext stackCtx = new(keyHash); + OperationStackContext stackCtx = new(keyHash); pendingContext.keyHash = keyHash; if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); - if (!FindTagAndTryTransientSLock(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (!FindTagAndTryEphemeralSLock(sessionFunctions, ref stackCtx, out var status)) return status; - stackCtx.SetRecordSourceToHashEntry(hlogBase); - // We have to assign a reference on declaration, so assign it here before we know whether LogicalAddress is above or below HeadAddress. - // It must be at this scope so it can be unlocked in 'finally'. - RecordInfo dummyRecordInfo = RecordInfo.InitialValid; - ref RecordInfo srcRecordInfo = ref dummyRecordInfo; + LogRecord srcLogRecord = default; ReadInfo readInfo = new() { @@ -80,22 +77,17 @@ internal OperationStatus InternalRead= hlogBase.SafeReadOnlyAddress) { // Mutable region (even fuzzy region is included here) - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); - pendingContext.recordInfo = srcRecordInfo; - readInfo.SetRecordInfo(ref srcRecordInfo); - - if (srcRecordInfo.IsClosedOrTombstoned(ref status)) + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); + if (srcLogRecord.Info.IsClosedOrTombstoned(ref status)) return status; - if (sessionFunctions.ConcurrentReader(ref key, ref input, ref stackCtx.recSrc.GetValue(), ref output, ref readInfo, ref srcRecordInfo)) - return OperationStatus.SUCCESS; - return CheckFalseActionStatus(readInfo); + return sessionFunctions.Reader(ref srcLogRecord, ref input, ref output, ref readInfo) + ? OperationStatus.SUCCESS + : CheckFalseActionStatus(ref readInfo); } if (stackCtx.recSrc.LogicalAddress >= hlogBase.HeadAddress) { // Immutable region - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); - pendingContext.recordInfo = srcRecordInfo; - readInfo.SetRecordInfo(ref srcRecordInfo); - - if (srcRecordInfo.IsClosedOrTombstoned(ref status)) + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); + if (srcLogRecord.Info.IsClosedOrTombstoned(ref status)) return status; - if (sessionFunctions.SingleReader(ref key, ref input, ref stackCtx.recSrc.GetValue(), ref output, ref readInfo)) + if (sessionFunctions.Reader(ref srcLogRecord, ref input, ref output, ref readInfo)) { - if (pendingContext.readCopyOptions.CopyFrom != ReadCopyFrom.AllImmutable) - return OperationStatus.SUCCESS; - return CopyFromImmutable(ref key, ref input, ref output, userContext, ref pendingContext, sessionFunctions, ref stackCtx, ref status, stackCtx.recSrc.GetValue()); + return pendingContext.readCopyOptions.CopyFrom != ReadCopyFrom.AllImmutable + ? OperationStatus.SUCCESS + : CopyFromImmutable(ref srcLogRecord, ref input, ref output, userContext, ref pendingContext, sessionFunctions, ref stackCtx, ref status); } - return CheckFalseActionStatus(readInfo); + return CheckFalseActionStatus(ref readInfo); } if (stackCtx.recSrc.LogicalAddress >= hlogBase.BeginAddress) { // On-Disk Region - Debug.Assert(!sessionFunctions.IsManualLocking || LockTable.IsLocked(ref stackCtx.hei), "A Lockable-session Read() of an on-disk key requires a LockTable lock"); + Debug.Assert(!sessionFunctions.IsTransactionalLocking || LockTable.IsLocked(ref stackCtx.hei), "A Transactional-session Read() of an on-disk key requires a LockTable lock"); // Note: we do not lock here; we wait until reading from disk, then lock in the ContinuePendingRead chain. if (hlogBase.IsNullDevice) return OperationStatus.NOTFOUND; - CreatePendingReadContext(ref key, ref input, output, userContext, ref pendingContext, sessionFunctions, stackCtx.recSrc.LogicalAddress); + CreatePendingReadContext(key, ref input, ref output, userContext, ref pendingContext, sessionFunctions, stackCtx.recSrc.LogicalAddress); return OperationStatus.RECORD_ON_DISK; } // No record found - Debug.Assert(!sessionFunctions.IsManualLocking || LockTable.IsLocked(ref stackCtx.hei), "A Lockable-session Read() of a non-existent key requires a LockTable lock"); + Debug.Assert(!sessionFunctions.IsTransactionalLocking || LockTable.IsLocked(ref stackCtx.hei), "A Transactional-session Read() of a non-existent key requires a LockTable lock"); return OperationStatus.NOTFOUND; } finally { stackCtx.HandleNewRecordOnException(this); - TransientSUnlock(sessionFunctions, ref key, ref stackCtx); + EphemeralSUnlock(sessionFunctions, ref stackCtx); } } // No AggressiveInlining; this is a less-common function and it may improve inlining of InternalRead to have this be a virtcall. - private OperationStatus CopyFromImmutable(ref TKey key, ref TInput input, ref TOutput output, TContext userContext, + private OperationStatus CopyFromImmutable(ref LogRecord srcLogRecord, ref TInput input, ref TOutput output, TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref OperationStatus status, TValue recordValue) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + ref OperationStackContext stackCtx, ref OperationStatus status) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { if (pendingContext.readCopyOptions.CopyTo == ReadCopyTo.MainLog) { - status = ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref key, ref input, ref recordValue, ref output, userContext, ref stackCtx, - WriteReason.CopyToTail, wantIO: false); - if (status == OperationStatus.ALLOCATE_FAILED && pendingContext.IsAsync) // May happen due to CopyToTailFromReadOnly - CreatePendingReadContext(ref key, ref input, output, userContext, ref pendingContext, sessionFunctions, stackCtx.recSrc.LogicalAddress); + status = ConditionalCopyToTail(sessionFunctions, ref pendingContext, ref srcLogRecord, ref stackCtx, wantIO: false); return status; } - if (pendingContext.readCopyOptions.CopyTo == ReadCopyTo.ReadCache - && TryCopyToReadCache(sessionFunctions, ref pendingContext, ref key, ref input, ref recordValue, ref stackCtx)) + if (pendingContext.readCopyOptions.CopyTo == ReadCopyTo.ReadCache && TryCopyToReadCache(ref srcLogRecord, sessionFunctions, ref pendingContext, ref stackCtx)) { // Copy to read cache is "best effort"; we don't return an error if it fails. return OperationStatus.SUCCESS | OperationStatus.COPIED_RECORD_TO_READ_CACHE; @@ -196,7 +178,7 @@ private OperationStatus CopyFromImmutable /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalReadAtAddress(long readAtAddress, ref TKey key, ref TInput input, ref TOutput output, + internal OperationStatus InternalReadAtAddress(long readAtAddress, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { if (readAtAddress < hlogBase.BeginAddress) return OperationStatus.NOTFOUND; - pendingContext.IsReadAtAddress = true; + pendingContext.SetIsReadAtAddress(); // We do things in a different order here than in InternalRead, in part to handle NoKey (especially with Revivification). if (readAtAddress < hlogBase.HeadAddress) { // Do not trace back in the pending callback if it is a key mismatch. - pendingContext.NoKey = true; + pendingContext.SetIsNoKey(); - CreatePendingReadContext(ref key, ref input, output, userContext, ref pendingContext, sessionFunctions, readAtAddress); + CreatePendingReadContext(key, ref input, ref output, userContext, ref pendingContext, sessionFunctions, readAtAddress); return OperationStatus.RECORD_ON_DISK; } // We're in-memory, so it is safe to get the address now. - var physicalAddress = hlog.GetPhysicalAddress(readAtAddress); + var srcLogRecord = hlog.CreateLogRecord(readAtAddress); - TKey defaultKey = default; + // Get the key hash. if (readOptions.KeyHash.HasValue) pendingContext.keyHash = readOptions.KeyHash.Value; - else if (!pendingContext.NoKey) - pendingContext.keyHash = storeFunctions.GetKeyHashCode64(ref key); + else if (!pendingContext.IsNoKey) + pendingContext.keyHash = storeFunctions.GetKeyHashCode64(key); else { // We have NoKey and an in-memory address so we must get the record to get the key to get the hashcode check for index growth, // possibly lock the bucket, etc. - pendingContext.keyHash = storeFunctions.GetKeyHashCode64(ref hlog.GetKey(physicalAddress)); - -#pragma warning disable CS9085 // "This ref-assigns a value that has a narrower escape scope than the target", but we don't return the reference. - // Note: With bucket-based locking the key is not used for Transient locks (only the key's hashcode is used). A key-based locking system - // would require this to be the actual key. We do *not* set this to the record key in case that is reclaimed by revivification. - key = ref defaultKey; -#pragma warning restore CS9085 + pendingContext.keyHash = storeFunctions.GetKeyHashCode64(srcLogRecord.Key); } - OperationStackContext stackCtx = new(pendingContext.keyHash); + OperationStackContext stackCtx = new(pendingContext.keyHash); if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); - if (!FindTagAndTryTransientSLock(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (!FindTagAndTryEphemeralSLock(sessionFunctions, ref stackCtx, out var status)) return status; stackCtx.SetRecordSourceToHashEntry(hlogBase); @@ -299,19 +275,15 @@ internal OperationStatus InternalReadAtAddress= hlogBase.SafeReadOnlyAddress) - { - // Mutable region (even fuzzy region is included here). - sessionFunctions.ConcurrentReader(ref stackCtx.recSrc.GetKey(), ref input, ref stackCtx.recSrc.GetValue(), ref output, ref readInfo, ref srcRecordInfo); - } - else - { - // Immutable region (we tested for < HeadAddress above). - sessionFunctions.SingleReader(ref stackCtx.recSrc.GetKey(), ref input, ref stackCtx.recSrc.GetValue(), ref output, ref readInfo); - } + _ = sessionFunctions.Reader(ref srcLogRecord, ref input, ref output, ref readInfo); } finally { stackCtx.HandleNewRecordOnException(this); - TransientSUnlock(sessionFunctions, ref key, ref stackCtx); + EphemeralSUnlock(sessionFunctions, ref stackCtx); } return status; } // No AggressiveInlining; this is called only for the pending case and may improve inlining of InternalRead in the normal case if the compiler decides not to inline this. - private void CreatePendingReadContext(ref TKey key, ref TInput input, TOutput output, TContext userContext, + private void CreatePendingReadContext(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, long logicalAddress) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { pendingContext.type = OperationType.READ; - if (!pendingContext.NoKey && pendingContext.key == default) // If this is true, we don't have a valid key - pendingContext.key = hlog.GetKeyContainer(ref key); - if (pendingContext.input == default) - pendingContext.input = sessionFunctions.GetHeapContainer(ref input); - - pendingContext.output = output; - sessionFunctions.ConvertOutputToHeap(ref input, ref pendingContext.output); - - pendingContext.userContext = userContext; + pendingContext.SerializeForReadOrRMW(key, ref input, ref output, userContext, sessionFunctions, hlogBase.bufferPool); pendingContext.logicalAddress = logicalAddress; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs index b0fd940682e..c0d19cfac8e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs @@ -1,23 +1,24 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// - /// Upsert operation. Replaces the value corresponding to 'key' with provided 'value', if one exists - /// else inserts a new record with 'key' and 'value'. + /// Upsert operation. Replaces the value corresponding to 'key' with provided 'value', if one exists, else inserts a new record with 'key' and 'value'. /// /// key of the record. /// /// input used to update the value. - /// value to be updated to (or inserted if key does not exist). + /// String value to be updated to (or inserted if key does not exist); exclusive with . + /// String value to be updated to (or inserted if key does not exist); exclusive with . /// output where the result of the update can be placed /// User context for the operation, in case it goes pending. /// Pending context used internally to store the context of the operation. @@ -43,29 +44,31 @@ public unsafe partial class TsavoriteKV /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalUpsert(ref TKey key, long keyHash, ref TInput input, ref TValue value, ref TOutput output, + internal OperationStatus InternalUpsert(ReadOnlySpan key, long keyHash, ref TInput input, + ReadOnlySpan srcStringValue, IHeapObject srcObjectValue, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref TContext userContext, ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TValueSelector : IUpsertValueSelector + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { var latchOperation = LatchOperation.None; - OperationStackContext stackCtx = new(keyHash); + OperationStackContext stackCtx = new(keyHash); pendingContext.keyHash = keyHash; if (sessionFunctions.Ctx.phase == Phase.IN_PROGRESS_GROW) SplitBuckets(stackCtx.hei.hash); - if (!FindOrCreateTagAndTryTransientXLock(sessionFunctions, ref key, ref stackCtx, out OperationStatus status)) + if (!FindOrCreateTagAndTryEphemeralXLock(sessionFunctions, ref stackCtx, out OperationStatus status)) return status; - RecordInfo dummyRecordInfo = RecordInfo.InitialValid; - ref RecordInfo srcRecordInfo = ref dummyRecordInfo; + LogRecord srcLogRecord = default; // We must use try/finally to ensure unlocking even in the presence of exceptions. try { // We blindly insert if the key isn't in the mutable region, so only check down to ReadOnlyAddress (minRevivifiableAddress is always >= ReadOnlyAddress). - if (!TryFindRecordForUpdate(ref key, ref stackCtx, hlogBase.ReadOnlyAddress, out status)) + if (!TryFindRecordForUpdate(key, ref stackCtx, hlogBase.ReadOnlyAddress, out status)) return status; // Note: Upsert does not track pendingContext.InitialAddress because we don't have an InternalContinuePendingUpsert @@ -73,7 +76,7 @@ internal OperationStatus InternalUpsert= hlogBase.ReadOnlyAddress) { - srcRecordInfo = ref stackCtx.recSrc.GetInfo(); + srcLogRecord = stackCtx.recSrc.CreateLogRecord(); // Mutable Region: Update the record in-place. We perform mutable updates only if we are in normal processing phase of checkpointing UpsertInfo upsertInfo = new() @@ -109,26 +112,27 @@ internal OperationStatus InternalUpsert= GetMinRevivifiableAddress()) { - if (TryRevivifyInChain(ref key, ref input, ref value, ref output, ref pendingContext, sessionFunctions, ref stackCtx, ref srcRecordInfo, ref upsertInfo, out status, ref recordValue) + if (TryRevivifyInChain( + ref srcLogRecord, ref input, srcStringValue, srcObjectValue, ref inputLogRecord, ref output, ref pendingContext, sessionFunctions, ref stackCtx, ref upsertInfo, out status) || status != OperationStatus.SUCCESS) goto LatchRelease; } goto CreateNewRecord; } - // upsertInfo's lengths are filled in and GetValueLengths and SetLength are called inside ConcurrentWriter. - if (sessionFunctions.ConcurrentWriter(stackCtx.recSrc.PhysicalAddress, ref key, ref input, ref value, ref recordValue, ref output, ref upsertInfo, ref srcRecordInfo)) + var sizeInfo = TValueSelector.GetUpsertRecordSize(hlog, srcLogRecord.Key, srcStringValue, srcObjectValue, ref inputLogRecord, ref input, sessionFunctions); + + // Type arg specification is needed because we don't pass TContext + var ok = TValueSelector.InPlaceWriter( + ref srcLogRecord, ref sizeInfo, ref input, srcStringValue, srcObjectValue, ref inputLogRecord, ref output, ref upsertInfo, sessionFunctions); + if (ok) { MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); - pendingContext.recordInfo = srcRecordInfo; pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); goto LatchRelease; @@ -139,34 +143,30 @@ internal OperationStatus InternalUpsert( + key, ref srcLogRecord, ref input, srcStringValue, srcObjectValue, ref inputLogRecord, ref output, ref pendingContext, sessionFunctions, ref stackCtx); + // We should never return "SUCCESS" for a new record operation: it returns NOTFOUND on success. + Debug.Assert(OperationStatusUtils.IsAppend(status) || OperationStatusUtils.BasicOpCode(status) != OperationStatus.SUCCESS); goto LatchRelease; } finally { stackCtx.HandleNewRecordOnException(this); - TransientXUnlock(sessionFunctions, ref key, ref stackCtx); + EphemeralXUnlock(sessionFunctions, ref stackCtx); } LatchRelease: @@ -187,75 +187,42 @@ internal OperationStatus InternalUpsert(ref TKey key, ref TInput input, ref TValue value, TOutput output, TContext userContext, - ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - { - pendingContext.type = OperationType.UPSERT; - if (pendingContext.key == default) - pendingContext.key = hlog.GetKeyContainer(ref key); - if (pendingContext.input == default) - pendingContext.input = sessionFunctions.GetHeapContainer(ref input); - if (pendingContext.value == default) - pendingContext.value = hlog.GetValueContainer(ref value); - - pendingContext.output = output; - sessionFunctions.ConvertOutputToHeap(ref input, ref pendingContext.output); - - pendingContext.userContext = userContext; - pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; - } - - private bool TryRevivifyInChain(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref PendingContext pendingContext, - TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo, ref UpsertInfo upsertInfo, - out OperationStatus status, ref TValue recordValue) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private bool TryRevivifyInChain(ref LogRecord logRecord, ref TInput input, + ReadOnlySpan srcStringValue, IHeapObject srcObjectValue, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref PendingContext pendingContext, + TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx, ref UpsertInfo upsertInfo, out OperationStatus status) + where TValueSelector : IUpsertValueSelector + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - if (IsFrozen(sessionFunctions, ref stackCtx, ref srcRecordInfo)) + if (IsFrozen(sessionFunctions, ref stackCtx, logRecord.Info)) goto NeedNewRecord; // This record is safe to revivify even if its PreviousAddress points to a valid record, because it is revivified for the same key. - var ok = true; + var ok = false; try { - if (srcRecordInfo.Tombstone) - { - srcRecordInfo.ClearTombstone(); + logRecord.ClearOptionals(); + logRecord.InfoRef.ClearTombstone(); - if (RevivificationManager.IsFixedLength) - upsertInfo.UsedValueLength = upsertInfo.FullValueLength = RevivificationManager.FixedValueLength; - else - { - var recordLengths = GetRecordLengths(stackCtx.recSrc.PhysicalAddress, ref recordValue, ref srcRecordInfo); - upsertInfo.FullValueLength = recordLengths.fullValueLength; + var sizeInfo = TValueSelector.GetUpsertRecordSize(hlog, logRecord.Key, srcStringValue, srcObjectValue, ref inputLogRecord, ref input, sessionFunctions); - // Input is not included in record-length calculations for Upsert - var (requiredSize, _, _) = hlog.GetRecordSize(ref key, ref value); - (ok, upsertInfo.UsedValueLength) = TryReinitializeTombstonedValue(sessionFunctions, - ref srcRecordInfo, ref key, ref recordValue, requiredSize, recordLengths); - } - - if (ok && sessionFunctions.SingleWriter(ref key, ref input, ref value, ref recordValue, ref output, ref upsertInfo, WriteReason.Upsert, ref srcRecordInfo)) - { - // Success - MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); - pendingContext.recordInfo = srcRecordInfo; - pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; - status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); - return true; - } - - // Did not revivify; restore the tombstone and leave the deleted record there. - srcRecordInfo.SetTombstone(); + // Type arg specification is needed because we don't pass TContext + ok = TValueSelector.InitialWriter( + ref logRecord, ref sizeInfo, ref input, srcStringValue, srcObjectValue, ref inputLogRecord, ref output, ref upsertInfo, sessionFunctions); + if (ok) + { + // Success + MarkPage(stackCtx.recSrc.LogicalAddress, sessionFunctions.Ctx); + pendingContext.logicalAddress = stackCtx.recSrc.LogicalAddress; + status = OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.InPlaceUpdatedRecord); + return true; } + // Did not revivify; restore the tombstone and leave the deleted record there. } finally { - if (ok) - SetExtraValueLength(ref recordValue, ref srcRecordInfo, upsertInfo.UsedValueLength, upsertInfo.FullValueLength); - else - SetTombstoneAndExtraValueLength(ref recordValue, ref srcRecordInfo, upsertInfo.UsedValueLength, upsertInfo.FullValueLength); // Restore tombstone and ensure default value on inability to update in place + if (!ok) + logRecord.InfoRef.SetTombstone(); } NeedNewRecord: @@ -264,7 +231,7 @@ private bool TryRevivifyInChain stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) + private LatchDestination CheckCPRConsistencyUpsert(Phase phase, ref OperationStackContext stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) { // See explanatory comments in CheckCPRConsistencyRMW. @@ -291,37 +258,40 @@ private LatchDestination CheckCPRConsistencyUpsert(Phase phase, ref OperationSta } /// - /// Create a new record for Upsert + /// Create a new record for Upsert from a source Key, Value, and Input /// /// The record Key + /// The source record, if . and + /// it is either too small or is in readonly region, or is in readcache /// Input to the operation - /// The value to insert - /// The result of ISessionFunctions.SingleWriter + /// String value to be set to; exclusive with and . + /// String value to be set to; exclusive with and . + /// Log record to be copied from; exclusive with and . + /// The result of ISessionFunctions operation /// Information about the operation context /// The current session - /// Contains the and structures for this operation, + /// Contains the and structures for this operation, /// and allows passing back the newLogicalAddress for invalidation in the case of exceptions. - /// If ., - /// this is the for - private OperationStatus CreateNewRecordUpsert(ref TKey key, ref TInput input, ref TValue value, ref TOutput output, - ref PendingContext pendingContext, TSessionFunctionsWrapper sessionFunctions, - ref OperationStackContext stackCtx, ref RecordInfo srcRecordInfo) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private OperationStatus CreateNewRecordUpsert(ReadOnlySpan key, ref LogRecord srcLogRecord, ref TInput input, + ReadOnlySpan srcStringValue, IHeapObject srcObjectValue, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref PendingContext pendingContext, + TSessionFunctionsWrapper sessionFunctions, ref OperationStackContext stackCtx) + where TValueSelector : IUpsertValueSelector + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - var (actualSize, allocatedSize, keySize) = hlog.GetUpsertRecordSize(ref key, ref value, ref input, sessionFunctions); + var sizeInfo = TValueSelector.GetUpsertRecordSize(hlog, key, srcStringValue, srcObjectValue, ref inputLogRecord, ref input, sessionFunctions); AllocateOptions allocOptions = new() { - Recycle = true, - ElideSourceRecord = stackCtx.recSrc.HasMainLogSrc && CanElide(sessionFunctions, ref stackCtx, ref srcRecordInfo) + recycle = true, + elideSourceRecord = stackCtx.recSrc.HasMainLogSrc && CanElide(sessionFunctions, ref stackCtx, srcLogRecord.Info) }; - if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, actualSize, ref allocatedSize, keySize, allocOptions, - out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status)) + if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, ref sizeInfo, allocOptions, out var newLogicalAddress, out var newPhysicalAddress, out var allocatedSize, out var status)) return status; - ref RecordInfo newRecordInfo = ref WriteNewRecordInfo(ref key, hlogBase, newPhysicalAddress, inNewVersion: sessionFunctions.Ctx.InNewVersion, stackCtx.recSrc.LatestLogicalAddress); - if (allocOptions.ElideSourceRecord) - newRecordInfo.PreviousAddress = srcRecordInfo.PreviousAddress; + var newLogRecord = WriteNewRecordInfo(key, hlogBase, newLogicalAddress, newPhysicalAddress, sessionFunctions.Ctx.InNewVersion, previousAddress: stackCtx.recSrc.LatestLogicalAddress); + if (allocOptions.elideSourceRecord) + newLogRecord.InfoRef.PreviousAddress = srcLogRecord.Info.PreviousAddress; stackCtx.SetNewRecord(newLogicalAddress); UpsertInfo upsertInfo = new() @@ -331,63 +301,62 @@ private OperationStatus CreateNewRecordUpsert( + ref newLogRecord, ref sizeInfo, ref input, srcStringValue, srcObjectValue, ref 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, newPhysicalAddress, allocatedSize, ref sessionFunctions.Ctx.RevivificationStats)) + if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, ref newLogRecord, ref sessionFunctions.Ctx.RevivificationStats)) stackCtx.ClearNewRecord(); else - stackCtx.SetNewRecordInvalid(ref newRecordInfo); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); if (upsertInfo.Action == UpsertAction.CancelOperation) return OperationStatus.CANCELED; return OperationStatus.NOTFOUND; // But not CreatedRecord } - SetExtraValueLength(ref newRecordValue, ref newRecordInfo, upsertInfo.UsedValueLength, upsertInfo.FullValueLength); - // Insert the new record by CAS'ing either directly into the hash entry or splicing into the readcache/mainlog boundary. // If the current record can be elided then we can freelist it; detach it by swapping its .PreviousAddress into newRecordInfo. - bool success = CASRecordIntoChain(ref key, ref stackCtx, newLogicalAddress, ref newRecordInfo); + success = CASRecordIntoChain(newLogicalAddress, ref newLogRecord, ref stackCtx); if (success) { - PostCopyToTail(ref key, ref stackCtx, ref srcRecordInfo); + PostCopyToTail(ref srcLogRecord, ref stackCtx); - sessionFunctions.PostSingleWriter(ref key, ref input, ref value, ref newRecordValue, ref output, ref upsertInfo, WriteReason.Upsert, ref newRecordInfo); + // Type arg specification is needed because we don't pass TContext + TValueSelector.PostInitialWriter( + ref newLogRecord, ref sizeInfo, ref input, srcStringValue, srcObjectValue, ref inputLogRecord, ref output, ref upsertInfo, sessionFunctions); // ElideSourceRecord means we have verified that the old source record is elidable and now that CAS has replaced it in the HashBucketEntry with // the new source record that does not point to the old source record, we have elided it, so try to transfer to freelist. - if (allocOptions.ElideSourceRecord) + if (allocOptions.elideSourceRecord) { // Success should always Seal the old record. This may be readcache, readonly, or the temporary recordInfo, which is OK and saves the cost of an "if". - srcRecordInfo.SealAndInvalidate(); // The record was elided, so Invalidate + srcLogRecord.InfoRef.SealAndInvalidate(); // The record was elided, so Invalidate if (stackCtx.recSrc.LogicalAddress >= GetMinRevivifiableAddress()) - { - // We need to re-get the old record's length because upsertInfo has the new record's info. If freelist-add fails, it remains Sealed/Invalidated. - var oldRecordLengths = GetRecordLengths(stackCtx.recSrc.PhysicalAddress, ref hlog.GetValue(stackCtx.recSrc.PhysicalAddress), ref srcRecordInfo); - TryTransferToFreeList(sessionFunctions, ref stackCtx, ref srcRecordInfo, oldRecordLengths); - } + _ = TryTransferToFreeList(sessionFunctions, ref stackCtx, ref srcLogRecord); + else + DisposeRecord(ref srcLogRecord, DisposeReason.Elided); } - else - srcRecordInfo.Seal(); // The record was not elided, so do not Invalidate + else if (stackCtx.recSrc.HasMainLogSrc) + srcLogRecord.InfoRef.Seal(); // The record was not elided, so do not Invalidate stackCtx.ClearNewRecord(); - pendingContext.recordInfo = newRecordInfo; pendingContext.logicalAddress = newLogicalAddress; return OperationStatusUtils.AdvancedOpCode(OperationStatus.NOTFOUND, StatusCode.CreatedRecord); } // CAS failed - stackCtx.SetNewRecordInvalid(ref newRecordInfo); - storeFunctions.DisposeRecord(ref hlog.GetKey(newPhysicalAddress), ref hlog.GetValue(newPhysicalAddress), DisposeReason.SingleWriterCASFailed); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); + DisposeRecord(ref newLogRecord, DisposeReason.InitialWriterCASFailed); - SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress, allocatedSize); + SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress); return OperationStatus.RETRY_NOW; // CAS failure does not require epoch refresh } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/ILockTable.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/ILockTable.cs index ec3b7c9df89..b8747b12b1a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/ILockTable.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/ILockTable.cs @@ -6,13 +6,12 @@ namespace Tsavorite.core { /// - /// Manual-enabled (both manual and transient) LockTable interface definition + /// LockTable interface definition (for both Transactional and Ephemeral) /// - /// - public interface ILockTable : IDisposable + public interface ILockTable : IDisposable { /// - /// Try to acquire a manual lock for the key. + /// Whether the lock table is enabled. /// public bool IsEnabled { get; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/OverflowBucketLockTable.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/OverflowBucketLockTable.cs index e3168adab1e..2504ec75442 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/OverflowBucketLockTable.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/OverflowBucketLockTable.cs @@ -7,17 +7,17 @@ namespace Tsavorite.core { - internal struct OverflowBucketLockTable : ILockTable - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal struct OverflowBucketLockTable : ILockTable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - private readonly TsavoriteKV store; + private readonly TsavoriteKV store; internal readonly long NumBuckets => store.state[store.resizeInfo.version].size_mask + 1; public readonly bool IsEnabled => store is not null; - internal OverflowBucketLockTable(TsavoriteKV store) => this.store = store; + internal OverflowBucketLockTable(TsavoriteKV store) => this.store = store; internal readonly long GetSize() => store.state[store.resizeInfo.version].size_mask; @@ -83,16 +83,16 @@ public unsafe LockState GetLockState(ref HashEntryInfo hei) IsLockedExclusive = HashBucket.IsLatchedExclusive(hei.firstBucket) }; - private static int KeyHashComparer(TLockableKey key1, TLockableKey key2, long size_mask) - where TLockableKey : ILockableKey + private static int KeyHashComparer(TTransactionalKey key1, TTransactionalKey key2, long size_mask) + where TTransactionalKey : ITransactionalKey { var idx1 = GetBucketIndex(key1.KeyHash, size_mask); var idx2 = GetBucketIndex(key2.KeyHash, size_mask); return (idx1 != idx2) ? idx1.CompareTo(idx2) : ((byte)key1.LockType).CompareTo((byte)key2.LockType); } - private static int KeyHashComparer(ref TLockableKey key1, ref TLockableKey key2, long size_mask) - where TLockableKey : ILockableKey + private static int KeyHashComparer(ref TTransactionalKey key1, ref TTransactionalKey key2, long size_mask) + where TTransactionalKey : ITransactionalKey { var idx1 = GetBucketIndex(key1.KeyHash, size_mask); var idx2 = GetBucketIndex(key2.KeyHash, size_mask); @@ -100,36 +100,36 @@ private static int KeyHashComparer(ref TLockableKey key1, ref TLoc } /// - internal int CompareKeyHashes(TLockableKey key1, TLockableKey key2) - where TLockableKey : ILockableKey + internal int CompareKeyHashes(TTransactionalKey key1, TTransactionalKey key2) + where TTransactionalKey : ITransactionalKey => KeyHashComparer(key1, key2, store.state[store.resizeInfo.version].size_mask); /// - internal int CompareKeyHashes(ref TLockableKey key1, ref TLockableKey key2) - where TLockableKey : ILockableKey + internal int CompareKeyHashes(ref TTransactionalKey key1, ref TTransactionalKey key2) + where TTransactionalKey : ITransactionalKey => KeyHashComparer(ref key1, ref key2, store.state[store.resizeInfo.version].size_mask); /// - internal void SortKeyHashes(TLockableKey[] keys) - where TLockableKey : ILockableKey - => Array.Sort(keys, new KeyComparer(store.state[store.resizeInfo.version].size_mask)); + internal void SortKeyHashes(TTransactionalKey[] keys) + where TTransactionalKey : ITransactionalKey + => Array.Sort(keys, new KeyComparer(store.state[store.resizeInfo.version].size_mask)); /// - internal void SortKeyHashes(TLockableKey[] keys, int start, int count) - where TLockableKey : ILockableKey - => Array.Sort(keys, start, count, new KeyComparer(store.state[store.resizeInfo.version].size_mask)); + internal void SortKeyHashes(TTransactionalKey[] keys, int start, int count) + where TTransactionalKey : ITransactionalKey + => Array.Sort(keys, start, count, new KeyComparer(store.state[store.resizeInfo.version].size_mask)); /// /// Need this struct because the Comparison{T} form of Array.Sort is not available with start and length arguments. /// - struct KeyComparer : IComparer - where TLockableKey : ILockableKey + struct KeyComparer : IComparer + where TTransactionalKey : ITransactionalKey { readonly long size_mask; internal KeyComparer(long s) => size_mask = s; - public int Compare(TLockableKey key1, TLockableKey key2) => KeyHashComparer(key1, key2, size_mask); + public int Compare(TTransactionalKey key1, TTransactionalKey key2) => KeyHashComparer(key1, key2, size_mask); } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/TransientLocking.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/TransientLocking.cs index 34ccc80660e..54427e103d7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/TransientLocking.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Locking/TransientLocking.cs @@ -1,22 +1,23 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool TryTransientXLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx, + private bool TryEphemeralXLock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx, out OperationStatus status) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - if (sessionFunctions.TryLockTransientExclusive(ref key, ref stackCtx)) + if (sessionFunctions.TryLockEphemeralExclusive(ref stackCtx)) { status = OperationStatus.SUCCESS; return true; @@ -26,21 +27,21 @@ private bool TryTransientXLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + private static void EphemeralXUnlock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - if (stackCtx.recSrc.HasTransientXLock) - sessionFunctions.UnlockTransientExclusive(ref key, ref stackCtx); + if (stackCtx.recSrc.HasEphemeralXLock) + sessionFunctions.UnlockEphemeralExclusive(ref stackCtx); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool TryTransientSLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx, + internal bool TryEphemeralSLock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx, out OperationStatus status) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - if (sessionFunctions.TryLockTransientShared(ref key, ref stackCtx)) + if (sessionFunctions.TryLockEphemeralShared(ref stackCtx)) { status = OperationStatus.SUCCESS; return true; @@ -50,36 +51,36 @@ internal bool TryTransientSLock(TSessionFunctionsWrapper sessionFunctions, ref TKey key, - ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal static void EphemeralSUnlock(TSessionFunctionsWrapper sessionFunctions, + ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - if (stackCtx.recSrc.HasTransientSLock) - sessionFunctions.UnlockTransientShared(ref key, ref stackCtx); + if (stackCtx.recSrc.HasEphemeralSLock) + sessionFunctions.UnlockEphemeralShared(ref stackCtx); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void LockForScan(ref OperationStackContext stackCtx, ref TKey key) + internal void LockForScan(ref OperationStackContext stackCtx, ReadOnlySpan key) { Debug.Assert(!stackCtx.recSrc.HasLock, $"Should not call LockForScan if recSrc already has a lock ({stackCtx.recSrc.LockStateString()})"); - // This will always be a transient lock as it is not session-based - stackCtx = new(storeFunctions.GetKeyHashCode64(ref key)); + // This will always be an Ephemeral lock as it is not session-based + stackCtx = new(storeFunctions.GetKeyHashCode64(key)); _ = FindTag(ref stackCtx.hei); stackCtx.SetRecordSourceToHashEntry(hlogBase); while (!LockTable.TryLockShared(ref stackCtx.hei)) epoch.ProtectAndDrain(); - stackCtx.recSrc.SetHasTransientSLock(); + stackCtx.recSrc.SetHasEphemeralSLock(); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void UnlockForScan(ref OperationStackContext stackCtx) + internal void UnlockForScan(ref OperationStackContext stackCtx) { - if (stackCtx.recSrc.HasTransientSLock) + if (stackCtx.recSrc.HasEphemeralSLock) { LockTable.UnlockShared(ref stackCtx.hei); - stackCtx.recSrc.ClearHasTransientSLock(); + stackCtx.recSrc.ClearHasEphemeralSLock(); } } } 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 b7aacffc3af..04301bb8126 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 @@ -1,14 +1,15 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// if reset is true it simply resets the modified bit for the key @@ -18,24 +19,23 @@ public unsafe partial class TsavoriteKVRecordInfo of the key for checkModified. /// Operation Type, whether it is reset or check [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalModifiedBitOperation(ref TKey key, out RecordInfo modifiedInfo, bool reset = true) + internal OperationStatus InternalModifiedBitOperation(ReadOnlySpan key, out RecordInfo modifiedInfo, bool reset = true) { Debug.Assert(epoch.ThisInstanceProtected()); - HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(ref key)); ; + HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(key)); ; #region Trace back for record in in-memory HybridLog _ = FindTag(ref hei); var logicalAddress = hei.Address; - var physicalAddress = hlog.GetPhysicalAddress(logicalAddress); if (logicalAddress >= hlogBase.HeadAddress) { - ref RecordInfo recordInfo = ref hlog.GetInfo(physicalAddress); - if (recordInfo.Invalid || !storeFunctions.KeysEqual(ref key, ref hlog.GetKey(physicalAddress))) + var logRecord = hlog.CreateLogRecord(logicalAddress); + if (logRecord.Info.Invalid || !storeFunctions.KeysEqual(key, logRecord.Key)) { - logicalAddress = recordInfo.PreviousAddress; - TraceBackForKeyMatch(ref key, logicalAddress, hlogBase.HeadAddress, out logicalAddress, out physicalAddress); + logicalAddress = logRecord.Info.PreviousAddress; + TraceBackForKeyMatch(key, logicalAddress, hlogBase.HeadAddress, out logicalAddress, out _); } } #endregion @@ -43,7 +43,7 @@ internal OperationStatus InternalModifiedBitOperation(ref TKey key, out RecordIn modifiedInfo = default; if (logicalAddress >= hlogBase.HeadAddress) { - ref RecordInfo recordInfo = ref hlog.GetInfo(physicalAddress); + ref var recordInfo = ref LogRecord.GetInfoRef(hlog.GetPhysicalAddress(logicalAddress)); if (reset) { if (!recordInfo.TryResetModifiedAtomic()) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/OperationStackContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/OperationStackContext.cs index bbe24ff24a4..d098c7f7aed 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/OperationStackContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/OperationStackContext.cs @@ -6,13 +6,13 @@ namespace Tsavorite.core { - public struct OperationStackContext - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public struct OperationStackContext + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Note: Cannot use ref fields because they are not supported before net7.0. internal HashEntryInfo hei; - internal RecordSource recSrc; + internal RecordSource recSrc; [MethodImpl(MethodImplOptions.AggressiveInlining)] internal OperationStackContext(long keyHash) => hei = new(keyHash); @@ -23,14 +23,14 @@ public struct OperationStackContext /// /// The TsavoriteKV's hlog [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SetRecordSourceToHashEntry(AllocatorBase srcLog) => recSrc.Set(hei.Address, srcLog); + internal void SetRecordSourceToHashEntry(AllocatorBase srcLog) => recSrc.Set(hei.Address, srcLog); /// /// Sets to the current ., which is the current address /// in the hash table. This is the same effect as calling . /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void UpdateRecordSourceToCurrentHashEntry(AllocatorBase hlog) + internal void UpdateRecordSourceToCurrentHashEntry(AllocatorBase hlog) { hei.SetToCurrent(); SetRecordSourceToHashEntry(hlog); @@ -69,7 +69,7 @@ internal void SetNewRecordInvalid(ref RecordInfo newRecordInfo) /// Called during InternalXxx 'finally' handler, to set the new record invalid if an exception or other error occurred. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void HandleNewRecordOnException(TsavoriteKV store) + internal void HandleNewRecordOnException(TsavoriteKV store) { if (newLogicalAddress != Constants.kInvalidAddress) { 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 11f099046f0..ed8e7a798d5 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 @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Diagnostics; using System.Runtime.CompilerServices; using static Tsavorite.core.Utility; @@ -8,12 +9,12 @@ namespace Tsavorite.core { // Partial file for readcache functions - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool FindInReadCache(ref TKey key, ref OperationStackContext stackCtx, long minAddress = Constants.kInvalidAddress, bool alwaysFindLatestLA = true) + internal bool FindInReadCache(ReadOnlySpan key, ref OperationStackContext stackCtx, long minAddress = Constants.kInvalidAddress, bool alwaysFindLatestLA = true) { Debug.Assert(UseReadCache, "Should not call FindInReadCache if !UseReadCache"); @@ -43,22 +44,27 @@ internal bool FindInReadCache(ref TKey key, ref OperationStackContext= minAddress && !stackCtx.recSrc.HasReadCacheSrc - && storeFunctions.KeysEqual(ref key, ref readcache.GetKey(stackCtx.recSrc.LowestReadCachePhysicalAddress))) + if (!recordInfo.Invalid && stackCtx.recSrc.LatestLogicalAddress >= minAddress && !stackCtx.recSrc.HasReadCacheSrc) { - // Keep these at the current readcache location; they'll be the caller's source record. - stackCtx.recSrc.LogicalAddress = stackCtx.recSrc.LowestReadCacheLogicalAddress; - stackCtx.recSrc.PhysicalAddress = stackCtx.recSrc.LowestReadCachePhysicalAddress; - stackCtx.recSrc.SetHasReadCacheSrc(); - stackCtx.recSrc.SetAllocator(readCacheBase); - - // Read() does not need to continue past the found record; updaters need to continue to find latestLogicalAddress and lowestReadCache*Address. - if (!alwaysFindLatestLA) - return true; + ReadOnlySpan keySpan = recordInfo.KeyIsInline + ? LogRecord.GetInlineKey(stackCtx.recSrc.LowestReadCachePhysicalAddress) + : readcache.CreateLogRecord(stackCtx.recSrc.LowestReadCacheLogicalAddress).Key; + if (storeFunctions.KeysEqual(key, keySpan)) + { + // Keep these at the current readcache location; they'll be the caller's source record. + stackCtx.recSrc.LogicalAddress = stackCtx.recSrc.LowestReadCacheLogicalAddress; + stackCtx.recSrc.PhysicalAddress = stackCtx.recSrc.LowestReadCachePhysicalAddress; + stackCtx.recSrc.SetHasReadCacheSrc(); + stackCtx.recSrc.SetAllocator(readCacheBase); + + // Read() does not need to continue past the found record; updaters need to continue to find latestLogicalAddress and lowestReadCache*Address. + if (!alwaysFindLatestLA) + return true; + } } // Update the leading LatestLogicalAddress to recordInfo.PreviousAddress, and if that is a main log record, break out. @@ -80,7 +86,7 @@ internal bool FindInReadCache(ref TKey key, ref OperationStackContext stackCtx) + bool ReadCacheNeedToWaitForEviction(ref OperationStackContext stackCtx) { if (stackCtx.recSrc.LatestLogicalAddress < readCacheBase.HeadAddress) { @@ -94,7 +100,7 @@ bool ReadCacheNeedToWaitForEviction(ref OperationStackContext stackCtx, long newLogicalAddress) + private bool SpliceIntoHashChainAtReadCacheBoundary(ReadOnlySpan key, ref OperationStackContext stackCtx, long newLogicalAddress) { // Splice into the gap of the last readcache/first main log entries. Debug.Assert(stackCtx.recSrc.LowestReadCacheLogicalAddress >= readCacheBase.ClosedUntilAddress, @@ -102,13 +108,13 @@ private bool SpliceIntoHashChainAtReadCacheBoundary(ref TKey key, ref OperationS // If the LockTable is enabled, then we either have an exclusive lock and thus cannot have a competing insert to the readcache, or we are doing a // Read() so we allow a momentary overlap of records because they're the same value (no update is being done). - ref RecordInfo rcri = ref readcache.GetInfo(stackCtx.recSrc.LowestReadCachePhysicalAddress); + ref var rcri = ref LogRecord.GetInfoRef(stackCtx.recSrc.LowestReadCachePhysicalAddress); return rcri.TryUpdateAddress(stackCtx.recSrc.LatestLogicalAddress, newLogicalAddress); } // Skip over all readcache records in this key's chain, advancing stackCtx.recSrc to the first non-readcache record we encounter. [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SkipReadCache(ref OperationStackContext stackCtx, out bool didRefresh) + internal void SkipReadCache(ref OperationStackContext stackCtx, out bool didRefresh) { Debug.Assert(UseReadCache, "Should not call SkipReadCache if !UseReadCache"); didRefresh = false; @@ -136,7 +142,7 @@ internal void SkipReadCache(ref OperationStackContextbucket_entries[index]; + var entry = (HashBucketEntry*)&bucket->bucket_entries[index]; if (0 == entry->word) continue; @@ -164,7 +170,7 @@ private void SkipReadCacheBucket(HashBucket* bucket) while (true) { - logicalAddress = readcache.GetInfo(physicalAddress).PreviousAddress; + logicalAddress = LogRecord.GetInfo(physicalAddress).PreviousAddress; entry->Address = logicalAddress; if (!entry->ReadCache) break; @@ -175,16 +181,16 @@ private void SkipReadCacheBucket(HashBucket* bucket) // Called after a readcache insert, to make sure there was no race with another session that added a main-log record at the same time. [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool EnsureNoNewMainLogRecordWasSpliced(ref TKey key, RecordSource recSrc, long highestSearchedAddress, ref OperationStatus failStatus) + private bool EnsureNoNewMainLogRecordWasSpliced(ReadOnlySpan key, RecordSource recSrc, long highestSearchedAddress, ref OperationStatus failStatus) { - bool success = true; - ref RecordInfo lowest_rcri = ref readcache.GetInfo(recSrc.LowestReadCachePhysicalAddress); + var success = true; + var lowest_rcri = LogRecord.GetInfo(recSrc.LowestReadCachePhysicalAddress); Debug.Assert(!IsReadCache(lowest_rcri.PreviousAddress), "lowest-rcri.PreviousAddress should be a main-log address"); if (lowest_rcri.PreviousAddress > highestSearchedAddress) { // Someone added a new record in the splice region. It won't be readcache; that would've been added at tail. See if it's our key. var minAddress = highestSearchedAddress > hlogBase.HeadAddress ? highestSearchedAddress : hlogBase.HeadAddress; - if (TraceBackForKeyMatch(ref key, lowest_rcri.PreviousAddress, minAddress + 1, out long prevAddress, out _)) + if (TraceBackForKeyMatch(key, lowest_rcri.PreviousAddress, minAddress + 1, out var prevAddress, out _)) success = false; else if (prevAddress > highestSearchedAddress && prevAddress < hlogBase.HeadAddress) { @@ -206,7 +212,7 @@ private bool EnsureNoNewMainLogRecordWasSpliced(ref TKey key, RecordSource key, ref HashEntryInfo hei, long highestReadCacheAddressChecked) { Debug.Assert(UseReadCache, "Should not call ReadCacheCheckTailAfterSplice if !UseReadCache"); @@ -217,9 +223,9 @@ private void ReadCacheCheckTailAfterSplice(ref TKey key, ref HashEntryInfo hei, // Traverse for the key above untilAddress (which may not be in the readcache if there were no readcache records when it was retrieved). while (entry.ReadCache && (entry.Address > untilEntry.Address || !untilEntry.ReadCache)) { - var physicalAddress = readcache.GetPhysicalAddress(entry.AbsoluteAddress); - ref RecordInfo recordInfo = ref readcache.GetInfo(physicalAddress); - if (!recordInfo.Invalid && storeFunctions.KeysEqual(ref key, ref readcache.GetKey(physicalAddress))) + var logRecord = readcache.CreateLogRecord(entry.AbsoluteAddress); + ref var recordInfo = ref logRecord.InfoRef; + if (!recordInfo.Invalid && storeFunctions.KeysEqual(key, logRecord.Key)) { recordInfo.SetInvalidAtomic(); return; @@ -235,7 +241,7 @@ private void ReadCacheCheckTailAfterSplice(ref TKey key, ref HashEntryInfo hei, void ReadCacheAbandonRecord(long physicalAddress) { // TODO: We currently don't save readcache allocations for retry, but we could - ref var ri = ref readcache.GetInfo(physicalAddress); + ref var ri = ref LogRecord.GetInfoRef(physicalAddress); ri.SetInvalid(); ri.PreviousAddress = Constants.kTempInvalidAddress; // Necessary for ReadCacheEvict, but cannot be kInvalidAddress or we have recordInfo.IsNull } @@ -245,9 +251,9 @@ internal void ReadCacheEvict(long rcLogicalAddress, long rcToLogicalAddress) // Iterate readcache entries in the range rcFrom/ToLogicalAddress, and remove them from the hash chain. while (rcLogicalAddress < rcToLogicalAddress) { - var rcPhysicalAddress = readcache.GetPhysicalAddress(rcLogicalAddress); - var (_, rcAllocatedSize) = readcache.GetRecordSize(rcPhysicalAddress); - var rcRecordInfo = readcache.GetInfo(rcPhysicalAddress); + var logRecord = new LogRecord(readcache.GetPhysicalAddress(rcLogicalAddress)); + var (_, rcAllocatedSize) = logRecord.GetInlineRecordSizes(); + var rcRecordInfo = logRecord.Info; // Check PreviousAddress for null to handle the info.IsNull() "partial record at end of page" case as well as readcache CAS failures // (such failed records are not in the hash chain, so we must not process them here). We do process other Invalid records here. @@ -267,8 +273,7 @@ internal void ReadCacheEvict(long rcLogicalAddress, long rcToLogicalAddress) Debug.Assert(!IsReadCache(rcRecordInfo.PreviousAddress) || AbsoluteAddress(rcRecordInfo.PreviousAddress) < rcLogicalAddress, "Invalid record ordering in readcache"); // Find the hash index entry for the key in the store's hash table. - ref TKey key = ref readcache.GetKey(rcPhysicalAddress); - HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(ref key)); + HashEntryInfo hei = new(storeFunctions.GetKeyHashCode64(logRecord.Key)); if (!FindTag(ref hei)) goto NextRecord; @@ -289,27 +294,27 @@ private void ReadCacheEvictChain(long rcToLogicalAddress, ref HashEntryInfo hei) // Traverse the chain of readcache entries for this key, looking "ahead" to .PreviousAddress to see if it is less than readcache.HeadAddress. // nextPhysicalAddress remains Constants.kInvalidAddress if hei.Address is < HeadAddress; othrwise, it is the lowest-address readcache record // remaining following this eviction, and its .PreviousAddress is updated to each lower record in turn until we hit a non-readcache record. - long nextPhysicalAddress = Constants.kInvalidAddress; + var nextPhysicalAddress = Constants.kInvalidAddress; HashBucketEntry entry = new() { word = hei.entry.word }; while (entry.ReadCache) { var la = entry.AbsoluteAddress; - var pa = readcache.GetPhysicalAddress(la); - ref RecordInfo ri = ref readcache.GetInfo(pa); + var logRecord = new LogRecord(readcache.GetPhysicalAddress(la)); + ref var recordInfo = ref logRecord.InfoRef; #if DEBUG // Due to collisions, we can compare the hash code *mask* (i.e. the hash bucket index), not the key var mask = state[resizeInfo.version].size_mask; var rc_mask = hei.hash & mask; - var pa_mask = storeFunctions.GetKeyHashCode64(ref readcache.GetKey(pa)) & mask; + var pa_mask = storeFunctions.GetKeyHashCode64(logRecord.Key) & mask; Debug.Assert(rc_mask == pa_mask, "The keyHash mask of the hash-chain ReadCache entry does not match the one obtained from the initial readcache address"); #endif // If the record's address is above the eviction range, leave it there and track nextPhysicalAddress. if (la >= rcToLogicalAddress) { - nextPhysicalAddress = pa; - entry.word = ri.PreviousAddress; + nextPhysicalAddress = logRecord.physicalAddress; + entry.word = recordInfo.PreviousAddress; continue; } @@ -317,16 +322,16 @@ private void ReadCacheEvictChain(long rcToLogicalAddress, ref HashEntryInfo hei) // (nextPhysicalAddress).PreviousAddress to (la).PreviousAddress. if (nextPhysicalAddress != Constants.kInvalidAddress) { - ref RecordInfo nextri = ref readcache.GetInfo(nextPhysicalAddress); - if (nextri.TryUpdateAddress(entry.Address, ri.PreviousAddress)) - ri.PreviousAddress = Constants.kTempInvalidAddress; // The record is no longer in the chain + ref var nextri = ref LogRecord.GetInfoRef(nextPhysicalAddress); + if (nextri.TryUpdateAddress(entry.Address, recordInfo.PreviousAddress)) + recordInfo.PreviousAddress = Constants.kTempInvalidAddress; // The record is no longer in the chain entry.word = nextri.PreviousAddress; continue; } // We are evicting the record whose address is in the hash bucket; unlink 'la' by setting the hash bucket to point to (la).PreviousAddress. - if (hei.TryCAS(ri.PreviousAddress)) - ri.PreviousAddress = Constants.kTempInvalidAddress; // The record is no longer in the chain + if (hei.TryCAS(recordInfo.PreviousAddress)) + recordInfo.PreviousAddress = Constants.kTempInvalidAddress; // The record is no longer in the chain else hei.SetToCurrent(); entry.word = hei.entry.word; 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 5b8e0b2b6c8..a0519800d74 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 @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System.Diagnostics; using System.Runtime.CompilerServices; using static Tsavorite.core.Utility; @@ -11,9 +12,9 @@ namespace Tsavorite.core /// operations, where "source" is a copy source for RMW and/or a locked record. This is passed to functions that create records, such as /// TsavoriteKV.CreateNewRecord*() or TsavoriteKV.InternalTryCopyToTail(), and to unlocking utilities. /// - internal struct RecordSource - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal struct RecordSource + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// If valid, this is the logical address of a record. As "source", it may be copied from for RMW or pending Reads, @@ -54,14 +55,14 @@ internal struct RecordSource /// /// If , this is the allocator base (hlog or readcache) that is in. /// - internal AllocatorBase AllocatorBase { get; private set; } + internal AllocatorBase AllocatorBase { get; private set; } struct InternalStates { internal const int None = 0; - internal const int TransientSLock = 0x0001; // LockTable - internal const int TransientXLock = 0x0002; // LockTable - internal const int LockBits = TransientSLock | TransientXLock; + internal const int EphemeralSLock = 0x0001; // LockTable + internal const int EphemeralXLock = 0x0002; // LockTable + internal const int LockBits = EphemeralSLock | EphemeralXLock; internal const int MainLogSrc = 0x0100; internal const int ReadCacheSrc = 0x0200; @@ -79,13 +80,13 @@ void append(int value, string name) if ((state & value) != 0) { if (sb.Length > 0) - sb.Append(", "); - sb.Append(name); + _ = sb.Append(", "); + _ = sb.Append(name); } } - append(TransientSLock, nameof(TransientSLock)); - append(TransientXLock, nameof(TransientXLock)); + append(EphemeralSLock, nameof(EphemeralSLock)); + append(EphemeralXLock, nameof(EphemeralXLock)); append(MainLogSrc, nameof(MainLogSrc)); append(ReadCacheSrc, nameof(ReadCacheSrc)); return sb.ToString(); @@ -95,22 +96,22 @@ void append(int value, string name) int internalState; /// - /// Set (and cleared) by caller to indicate whether we have a LockTable-based Transient Shared lock (does not include Manual locks; this is per-operation only). + /// Set (and cleared) by caller to indicate whether we have a LockTable-based Ephemeral Shared lock (does not include Manual locks; this is per-operation only). /// - internal readonly bool HasTransientSLock => (internalState & InternalStates.TransientSLock) != 0; + internal readonly bool HasEphemeralSLock => (internalState & InternalStates.EphemeralSLock) != 0; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SetHasTransientSLock() => internalState |= InternalStates.TransientSLock; + internal void SetHasEphemeralSLock() => internalState |= InternalStates.EphemeralSLock; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void ClearHasTransientSLock() => internalState &= ~InternalStates.TransientSLock; + internal void ClearHasEphemeralSLock() => internalState &= ~InternalStates.EphemeralSLock; /// - /// Set (and cleared) by caller to indicate whether we have a LockTable-based Transient Exclusive lock (does not include Manual locks; this is per-operation only). + /// Set (and cleared) by caller to indicate whether we have a LockTable-based Ephemeral Exclusive lock (does not include Manual locks; this is per-operation only). /// - internal readonly bool HasTransientXLock => (internalState & InternalStates.TransientXLock) != 0; + internal readonly bool HasEphemeralXLock => (internalState & InternalStates.EphemeralXLock) != 0; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SetHasTransientXLock() => internalState |= InternalStates.TransientXLock; + internal void SetHasEphemeralXLock() => internalState |= InternalStates.EphemeralXLock; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void ClearHasTransientXLock() => internalState &= ~InternalStates.TransientXLock; + internal void ClearHasEphemeralXLock() => internalState &= ~InternalStates.EphemeralXLock; /// /// Indicates whether we have any type of non-Manual lock. @@ -138,11 +139,16 @@ void append(int value, string name) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal long SetPhysicalAddress() => PhysicalAddress = Allocator.GetPhysicalAddress(LogicalAddress); [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal readonly ref RecordInfo GetInfo() => ref Allocator.GetInfo(PhysicalAddress); - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal readonly ref TKey GetKey() => ref Allocator.GetKey(PhysicalAddress); + internal readonly ref RecordInfo GetInfoRef() => ref LogRecord.GetInfoRef(PhysicalAddress); + internal readonly RecordInfo GetInfo() => LogRecord.GetInfoRef(PhysicalAddress); + [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal readonly ref TValue GetValue() => ref Allocator.GetValue(PhysicalAddress); + internal readonly LogRecord CreateLogRecord() + { + Debug.Assert(PhysicalAddress != 0, "Cannot CreateLogRecord until PhysicalAddress is set"); + Debug.Assert(HasInMemorySrc, "Can only create a LogRecord for a record in main log memory"); + return Allocator.CreateLogRecord(LogicalAddress, PhysicalAddress); + } internal readonly bool HasInMemorySrc => (internalState & (InternalStates.MainLogSrc | InternalStates.ReadCacheSrc)) != 0; @@ -150,7 +156,7 @@ void append(int value, string name) /// Initialize to the latest logical address from the caller. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void Set(long latestLogicalAddress, AllocatorBase srcAllocatorBase) + internal void Set(long latestLogicalAddress, AllocatorBase srcAllocatorBase) { PhysicalAddress = default; LowestReadCacheLogicalAddress = default; @@ -158,17 +164,17 @@ internal void Set(long latestLogicalAddress, AllocatorBase srcAllocatorBase) + internal void SetAllocator(AllocatorBase srcAllocatorBase) { - this.AllocatorBase = srcAllocatorBase; - this.Allocator = AllocatorBase._wrapper; + AllocatorBase = srcAllocatorBase; + Allocator = AllocatorBase._wrapper; } [MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/CheckEmptyWorker.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/CheckEmptyWorker.cs index 8d3bfca58b3..2d6546c9773 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/CheckEmptyWorker.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/CheckEmptyWorker.cs @@ -8,9 +8,9 @@ namespace Tsavorite.core { - internal sealed class CheckEmptyWorker - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class CheckEmptyWorker + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // State control variables. internal struct State @@ -33,9 +33,9 @@ internal static string ToString(long state) CancellationTokenSource cts = new(); - readonly FreeRecordPool recordPool; + readonly FreeRecordPool recordPool; - internal CheckEmptyWorker(FreeRecordPool recordPool) => this.recordPool = recordPool; + internal CheckEmptyWorker(FreeRecordPool recordPool) => this.recordPool = recordPool; [MethodImpl(MethodImplOptions.AggressiveInlining)] internal unsafe void Start() 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 92cea4dacf1..9c604bc0c3e 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 @@ -14,7 +14,8 @@ namespace Tsavorite.core [StructLayout(LayoutKind.Explicit, Size = sizeof(long))] internal struct FreeRecord { - internal const int kSizeBits = 64 - RecordInfo.kPreviousAddressBits; + internal const int kSizeBits = 64 - RecordInfo.kPreviousAddressBits; // 16 + const int kSizeShiftInWord = RecordInfo.kPreviousAddressBits; const long kSizeMask = RevivificationBin.MaxInlineRecordSize - 1; @@ -29,12 +30,14 @@ internal struct FreeRecord internal const int StructSize = sizeof(long); + /// LogicalAddress of the record. public long Address { readonly get => word & RecordInfo.kPreviousAddressMaskInWord; set => word = (word & ~RecordInfo.kPreviousAddressMaskInWord) | (value & RecordInfo.kPreviousAddressMaskInWord); } + /// Inline size of the record. May contain overflow allocations. public readonly int Size => (int)((word & kSizeMaskInWord) >> kSizeShiftInWord); /// @@ -62,9 +65,9 @@ void SetEmptyAtomic(long oldWord) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool TryPeek(long recordSize, TsavoriteKV store, bool oversize, long minAddress, out int thisRecordSize) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal bool TryPeek(long recordSize, TsavoriteKV store, bool oversize, long minAddress, out int thisRecordSize) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { FreeRecord oldRecord = this; thisRecordSize = 0; @@ -75,6 +78,7 @@ internal bool TryPeek(long recordSize SetEmptyAtomic(oldRecord.word); return false; } + var thisSize = oversize ? GetRecordSize(store, oldRecord.Address) : oldRecord.Size; if (thisSize < recordSize) return false; @@ -107,7 +111,7 @@ internal readonly void MergeTo(ref RevivificationStats revivStats) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal bool TryTake(int recordSize, long minAddress, out long address, ref TakeResult takeResult) + internal bool TryTake(ref RecordSizeInfo sizeInfo, long minAddress, out long address, ref TakeResult takeResult) { address = 0; @@ -121,7 +125,7 @@ internal bool TryTake(int recordSize, long minAddress, out long address, ref Tak return false; else takeResult.addressOk = true; - if (oldRecord.Size < recordSize) + if (oldRecord.Size < sizeInfo.ActualInlineRecordSize) return false; else takeResult.recordSizeOk = true; @@ -139,19 +143,19 @@ internal bool TryTake(int recordSize, long minAddress, out long address, ref Tak } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static int GetRecordSize(TsavoriteKV store, long logicalAddress) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + private static int GetRecordSize(TsavoriteKV store, long logicalAddress) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - // Because this is oversize, we need hlog to get the length out of the record's value (it won't fit in FreeRecord.kSizeBits) - long physicalAddress = store.hlog.GetPhysicalAddress(logicalAddress); - return store.GetFreeRecordSize(physicalAddress, ref store.hlog.GetInfo(physicalAddress)); + // This is called for oversize, so we need hlog to get the length out of the record's value (it won't fit in FreeRecord.kSizeBits) + var logRecord = store.hlog.CreateLogRecord(logicalAddress); + return logRecord.GetInlineRecordSizes().allocatedSize; } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal unsafe bool TryTakeOversize(long recordSize, long minAddress, TsavoriteKV store, out long address, ref TakeResult takeResult) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal unsafe bool TryTakeOversize(ref RecordSizeInfo sizeInfo, long minAddress, TsavoriteKV store, out long address, ref TakeResult takeResult) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { address = 0; @@ -169,10 +173,8 @@ internal unsafe bool TryTakeOversize( takeResult.addressOk = true; // Because this is oversize, we need hlog to get the length out of the record's value (it won't fit in FreeRecord.kSizeBits) - long physicalAddress = store.hlog.GetPhysicalAddress(oldRecord.Address); - long thisSize = store.GetFreeRecordSize(physicalAddress, ref store.hlog.GetInfo(physicalAddress)); - - if (thisSize < recordSize) + long thisSize = GetRecordSize(store, oldRecord.Address); + if (thisSize < sizeInfo.ActualInlineRecordSize) return false; else takeResult.recordSizeOk = true; @@ -219,19 +221,19 @@ public override string ToString() return $"isEmpty {isEmpty}, recSizes {minRecordSize}..{maxRecordSize}, recSizeInc {segmentRecordSizeIncrement}, #recs {recordCount}; segments: segSize {segmentSize}, #segs {segmentCount}; scanLimit {scanStr}"; } - internal FreeRecordBin(ref RevivificationBin binDef, int prevBinRecordSize, bool isFixedLength) + internal FreeRecordBin(ref RevivificationBin binDef, int prevBinRecordSize) { // If the record size range is too much for the number of records in the bin, we must allow multiple record sizes per segment. // prevBinRecordSize is already verified to be a multiple of 8. var bindefRecordSize = RoundUp(binDef.RecordSize, 8); - if (isFixedLength || bindefRecordSize == prevBinRecordSize + 8) + if (bindefRecordSize == prevBinRecordSize + 8) { bestFitScanLimit = RevivificationBin.UseFirstFit; segmentSize = RoundUp(binDef.NumberOfRecords, MinSegmentSize); segmentCount = 1; segmentRecordSizeIncrement = 1; // For the division and multiplication in GetSegmentStart - minRecordSize = maxRecordSize = isFixedLength ? prevBinRecordSize : bindefRecordSize; + minRecordSize = maxRecordSize = bindefRecordSize; } else { @@ -270,8 +272,7 @@ internal FreeRecordBin(ref RevivificationBin binDef, int prevBinRecordSize, bool [MethodImpl(MethodImplOptions.AggressiveInlining)] internal int GetSegmentStart(int recordSize) { - // recordSize and segmentSizeIncrement are rounded up to 8, unless IsFixedLength in which case segmentSizeIncrement is 1. - // sizeOffset will be negative if we are searching the next-highest bin. + // recordSize and segmentSizeIncrement are rounded up to 8. sizeOffset will be negative if we are searching the next-highest bin. var sizeOffset = recordSize - minRecordSize; if (sizeOffset < 0) sizeOffset = 0; @@ -284,16 +285,16 @@ internal int GetSegmentStart(int recordSize) private FreeRecord* GetRecord(int recordIndex) => records + (recordIndex >= recordCount ? recordIndex - recordCount : recordIndex); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryAdd(long address, int recordSize, TsavoriteKV store, long minAddress, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public bool TryAdd(long logicalAddress, int recordSize, TsavoriteKV store, long minAddress, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { var segmentStart = GetSegmentStart(recordSize); for (var ii = 0; ii < recordCount; ++ii) { FreeRecord* record = GetRecord(segmentStart + ii); - if (record->Set(address, recordSize, minAddress)) + if (record->Set(logicalAddress, recordSize, minAddress)) { ++revivStats.successfulAdds; isEmpty = false; @@ -305,15 +306,15 @@ public bool TryAdd(long address, int } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTake(int recordSize, long minAddress, TsavoriteKV store, out long address, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - => TryTake(recordSize, minAddress, store, oversize: false, out address, ref revivStats); + public bool TryTake(ref RecordSizeInfo sizeInfo, long minAddress, TsavoriteKV store, out long address, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + => TryTake(ref sizeInfo, minAddress, store, oversize: false, out address, ref revivStats); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTake(int recordSize, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public bool TryTake(ref RecordSizeInfo sizeInfo, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { if (isEmpty) { @@ -321,16 +322,16 @@ public bool TryTake(int recordSize, l return false; } return (bestFitScanLimit == RevivificationBin.UseFirstFit) - ? TryTakeFirstFit(recordSize, minAddress, store, oversize, out address, ref revivStats) - : TryTakeBestFit(recordSize, minAddress, store, oversize, out address, ref revivStats); + ? TryTakeFirstFit(ref sizeInfo, minAddress, store, oversize, out address, ref revivStats) + : TryTakeBestFit(ref sizeInfo, minAddress, store, oversize, out address, ref revivStats); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTakeFirstFit(int recordSize, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public bool TryTakeFirstFit(ref RecordSizeInfo sizeInfo, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - var segmentStart = GetSegmentStart(recordSize); + var segmentStart = GetSegmentStart(sizeInfo.ActualInlineRecordSize); int retryCount = recordCount; FreeRecord.TakeResult takeResult = new(); @@ -339,7 +340,7 @@ public bool TryTakeFirstFit(int recor for (var ii = 0; ii < recordCount; ++ii) { FreeRecord* record = GetRecord(segmentStart + ii); - if (oversize ? record->TryTakeOversize(recordSize, minAddress, store, out address, ref takeResult) : record->TryTake(recordSize, minAddress, out address, ref takeResult)) + if (oversize ? record->TryTakeOversize(ref sizeInfo, minAddress, store, out address, ref takeResult) : record->TryTake(ref sizeInfo, minAddress, out address, ref takeResult)) { takeResult.MergeTo(ref revivStats); return true; @@ -355,13 +356,13 @@ public bool TryTakeFirstFit(int recor } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTakeBestFit(int recordSize, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public bool TryTakeBestFit(ref RecordSizeInfo sizeInfo, long minAddress, TsavoriteKV store, bool oversize, out long address, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Retry as long as we find a candidate, but reduce the best fit scan limit each retry. int localBestFitScanLimit = bestFitScanLimit; - var segmentStart = GetSegmentStart(recordSize); + var segmentStart = GetSegmentStart(sizeInfo.ActualInlineRecordSize); FreeRecord.TakeResult takeResult = new(); while (true) @@ -375,7 +376,7 @@ public bool TryTakeBestFit(int record { // For best-fit we must peek first without taking. record = GetRecord(segmentStart + ii); - if (record->TryPeek(recordSize, store, oversize, minAddress, out var thisRecordSize)) + if (record->TryPeek(sizeInfo.ActualInlineRecordSize, store, oversize, minAddress, out var thisRecordSize)) { bestFitIndex = ii; // Found exact match break; @@ -400,7 +401,7 @@ record = GetRecord(segmentStart + ii); } record = GetRecord(segmentStart + bestFitIndex); - if (oversize ? record->TryTakeOversize(recordSize, minAddress, store, out address, ref takeResult) : record->TryTake(recordSize, minAddress, out address, ref takeResult)) + if (oversize ? record->TryTakeOversize(ref sizeInfo, minAddress, store, out address, ref takeResult) : record->TryTake(ref sizeInfo, minAddress, out address, ref takeResult)) { takeResult.MergeTo(ref revivStats); return true; @@ -409,14 +410,14 @@ record = GetRecord(segmentStart + bestFitIndex); // We found a candidate but CAS failed. Reduce the best fit scan length and continue. localBestFitScanLimit /= 2; if (localBestFitScanLimit <= 1) - return TryTakeFirstFit(recordSize, minAddress, store, oversize, out address, ref revivStats); + return TryTakeFirstFit(ref sizeInfo, minAddress, store, oversize, out address, ref revivStats); } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void ScanForEmpty(FreeRecordPool recordPool, CancellationToken cancellationToken) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal void ScanForEmpty(FreeRecordPool recordPool, CancellationToken cancellationToken) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // Add() always sets isEmpty to false and we do not clear isEmpty on Take() because that could lead to more lost "isEmpty = false". // So this routine is called only if the bin is marked not-empty. @@ -437,40 +438,31 @@ internal void ScanForEmpty(FreeRecord } } - internal unsafe class FreeRecordPool : IDisposable - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal unsafe class FreeRecordPool : IDisposable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - internal readonly TsavoriteKV store; + internal readonly TsavoriteKV store; internal readonly FreeRecordBin[] bins; internal int numberOfBinsToSearch; - internal bool IsFixedLength; internal readonly int[] sizeIndexArray; private readonly int* sizeIndex; private readonly int numBins; - internal readonly CheckEmptyWorker checkEmptyWorker; + internal readonly CheckEmptyWorker checkEmptyWorker; /// public override string ToString() - => $"isFixedLen {IsFixedLength}, numBins {numBins}, searchNextBin {numberOfBinsToSearch}, checkEmptyWorker: {checkEmptyWorker}"; + => $"numBins {numBins}, searchNextBin {numberOfBinsToSearch}, checkEmptyWorker: {checkEmptyWorker}"; - internal FreeRecordPool(TsavoriteKV store, RevivificationSettings settings, int fixedRecordLength) + internal FreeRecordPool(TsavoriteKV store, RevivificationSettings settings) { this.store = store; - IsFixedLength = fixedRecordLength > 0; checkEmptyWorker = new(this); - if (IsFixedLength) - { - numBins = 1; - bins = [new FreeRecordBin(ref settings.FreeRecordBins[0], fixedRecordLength, isFixedLength: true)]; - return; - } - // First create the "size index": a cache-aligned vector of int bin sizes. This way searching for the bin // for a record size will stay in a single cache line (unless there are more than 16 bins). var sizeIndexCount = RoundUp(settings.FreeRecordBins.Length * sizeof(int), Constants.kCacheLineBytes) / sizeof(int); @@ -490,7 +482,7 @@ internal FreeRecordPool(TsavoriteKV s { if (prevBinRecordSize >= settings.FreeRecordBins[ii].RecordSize) continue; - FreeRecordBin bin = new(ref settings.FreeRecordBins[ii], prevBinRecordSize, isFixedLength: false); + FreeRecordBin bin = new(ref settings.FreeRecordBins[ii], prevBinRecordSize); sizeIndex[binList.Count] = bin.maxRecordSize; binList.Add(bin); prevBinRecordSize = bin.maxRecordSize; @@ -503,8 +495,6 @@ internal FreeRecordPool(TsavoriteKV s [MethodImpl(MethodImplOptions.AggressiveInlining)] internal bool GetBinIndex(int size, out int binIndex) { - Debug.Assert(!IsFixedLength, "Should only search bins if !IsFixedLength"); - // Sequential search in the sizeIndex for the requested size. for (var ii = 0; ii < numBins; ++ii) { @@ -519,13 +509,13 @@ internal bool GetBinIndex(int size, out int binIndex) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryAdd(long logicalAddress, int size, ref RevivificationStats revivStats) + private bool TryAddToBin(long logicalAddress, ref LogRecord logRecord, ref RevivificationStats revivStats) { var minAddress = store.GetMinRevivifiableAddress(); - int binIndex = 0; - if (logicalAddress < minAddress || (!IsFixedLength && !GetBinIndex(size, out binIndex))) + var recordSize = logRecord.GetInlineRecordSizes().allocatedSize; + if (logicalAddress < minAddress || (!GetBinIndex(recordSize, out var binIndex))) return false; - if (!bins[binIndex].TryAdd(logicalAddress, size, store, minAddress, ref revivStats)) + if (!bins[binIndex].TryAdd(logicalAddress, recordSize, store, minAddress, ref revivStats)) return false; // We've added a record, so now start the worker thread that periodically checks to see if Take() has emptied the bins. @@ -534,7 +524,7 @@ public bool TryAdd(long logicalAddress, int size, ref RevivificationStats revivS } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryAdd(long logicalAddress, long physicalAddress, int allocatedSize, ref RevivificationStats revivStats) + public bool TryAdd(long logicalAddress, ref LogRecord logRecord, ref RevivificationStats revivStats) { var minAddress = store.GetMinRevivifiableAddress(); if (logicalAddress < minAddress) @@ -542,10 +532,8 @@ public bool TryAdd(long logicalAddress, long physicalAddress, int allocatedSize, ++revivStats.failedAdds; return false; } - var recordInfo = store.hlog.GetInfo(physicalAddress); - recordInfo.TrySeal(invalidate: true); - store.SetFreeRecordSize(physicalAddress, ref recordInfo, allocatedSize); - bool result = TryAdd(logicalAddress, allocatedSize, ref revivStats); + logRecord.InfoRef.TrySeal(invalidate: true); + bool result = TryAddToBin(logicalAddress, ref logRecord, ref revivStats); if (result) ++revivStats.successfulAdds; @@ -555,18 +543,16 @@ public bool TryAdd(long logicalAddress, long physicalAddress, int allocatedSize, } [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTake(int recordSize, long minAddress, out long address, ref RevivificationStats revivStats) + public bool TryTake(ref RecordSizeInfo sizeInfo, long minAddress, out long address, ref RevivificationStats revivStats) { address = 0; bool result = false; - if (IsFixedLength) - result = bins[0].TryTake(recordSize, minAddress, store, out address, ref revivStats); - else if (GetBinIndex(recordSize, out int index)) + if (GetBinIndex(sizeInfo.ActualInlineRecordSize, out int index)) { // Try to Take from the initial bin and if unsuccessful, try the next-highest bin if requested. - result = bins[index].TryTake(recordSize, minAddress, store, oversize: sizeIndex[index] > RevivificationBin.MaxInlineRecordSize, out address, ref revivStats); + result = bins[index].TryTake(ref sizeInfo, minAddress, store, oversize: sizeIndex[index] > RevivificationBin.MaxInlineRecordSize, out address, ref revivStats); for (int ii = 0; !result && ii < numberOfBinsToSearch && index < numBins - 1; ++ii) - result = bins[++index].TryTake(recordSize, minAddress, store, oversize: sizeIndex[index] > RevivificationBin.MaxInlineRecordSize, out address, ref revivStats); + result = bins[++index].TryTake(ref sizeInfo, minAddress, store, oversize: sizeIndex[index] > RevivificationBin.MaxInlineRecordSize, out address, ref revivStats); } if (result) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RecordLengths.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RecordLengths.cs deleted file mode 100644 index b3db23ea4b4..00000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RecordLengths.cs +++ /dev/null @@ -1,233 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Diagnostics; -using System.Runtime.CompilerServices; - -namespace Tsavorite.core -{ - using static Utility; - - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal long GetMinRevivifiableAddress() - => RevivificationManager.GetMinRevivifiableAddress(hlogBase.GetTailAddress(), hlogBase.ReadOnlyAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static int GetValueOffset(long physicalAddress, ref TValue recordValue) => (int)((long)Unsafe.AsPointer(ref recordValue) - physicalAddress); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static unsafe int* GetExtraValueLengthPointer(ref TValue recordValue, int usedValueLength) - { - Debug.Assert(RoundUp(usedValueLength, sizeof(int)) == usedValueLength, "GetLiveFullValueLengthPointer: usedValueLength should have int-aligned length"); - return (int*)((long)Unsafe.AsPointer(ref recordValue) + usedValueLength); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal unsafe void SetExtraValueLength(ref TValue recordValue, ref RecordInfo recordInfo, int usedValueLength, int fullValueLength) - { - if (RevivificationManager.IsFixedLength) - recordInfo.ClearHasFiller(); - else - SetVarLenExtraValueLength(ref recordValue, ref recordInfo, usedValueLength, fullValueLength); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static unsafe void SetVarLenExtraValueLength(ref TValue recordValue, ref RecordInfo recordInfo, int usedValueLength, int fullValueLength) - { - usedValueLength = RoundUp(usedValueLength, sizeof(int)); - Debug.Assert(fullValueLength >= usedValueLength, $"SetFullValueLength: usedValueLength {usedValueLength} cannot be > fullValueLength {fullValueLength}"); - int extraValueLength = fullValueLength - usedValueLength; - if (extraValueLength >= sizeof(int)) - { - var extraValueLengthPtr = GetExtraValueLengthPointer(ref recordValue, usedValueLength); - Debug.Assert(*extraValueLengthPtr == 0 || *extraValueLengthPtr == extraValueLength, "existing ExtraValueLength should be 0 or the same value"); - - // We always store the "extra" as the difference between the aligned usedValueLength and the fullValueLength. - // However, the UpdateInfo structures use the unaligned usedValueLength; aligned usedValueLength is not visible to the user. - *extraValueLengthPtr = extraValueLength; - recordInfo.SetHasFiller(); - return; - } - recordInfo.ClearHasFiller(); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal (int usedValueLength, int fullValueLength, int fullRecordLength) GetRecordLengths(long physicalAddress, ref TValue recordValue, ref RecordInfo recordInfo) - { - // FixedLen may be GenericAllocator which does not point physicalAddress to the actual record location, so calculate fullRecordLength via GetAverageRecordSize(). - if (RevivificationManager.IsFixedLength) - return (RevivificationManager.FixedValueLength, RevivificationManager.FixedValueLength, hlog.GetAverageRecordSize()); - - int usedValueLength, fullValueLength, allocatedSize, valueOffset = GetValueOffset(physicalAddress, ref recordValue); - if (recordInfo.HasFiller) - { - usedValueLength = hlog.GetValueLength(ref recordValue); - var alignedUsedValueLength = RoundUp(usedValueLength, sizeof(int)); - fullValueLength = alignedUsedValueLength + *GetExtraValueLengthPointer(ref recordValue, alignedUsedValueLength); - Debug.Assert(fullValueLength >= usedValueLength, $"GetLengthsFromFiller: fullValueLength {fullValueLength} should be >= usedValueLength {usedValueLength}"); - allocatedSize = valueOffset + fullValueLength; - } - else - { - // Live VarLen record with no stored sizes; we always have a Key and Value (even if defaults). Return the full record length (including recordInfo and Key). - (int actualSize, allocatedSize) = hlog.GetRecordSize(physicalAddress); - usedValueLength = actualSize - valueOffset; - fullValueLength = allocatedSize - valueOffset; - } - - Debug.Assert(usedValueLength >= 0, $"GetLiveRecordLengths: usedValueLength {usedValueLength}"); - Debug.Assert(fullValueLength >= 0, $"GetLiveRecordLengths: fullValueLength {fullValueLength}"); - Debug.Assert(allocatedSize >= 0, $"GetLiveRecordLengths: fullRecordLength {allocatedSize}"); - return (usedValueLength, fullValueLength, allocatedSize); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private (int usedValueLength, int fullValueLength) GetNewValueLengths(int actualSize, int allocatedSize, long newPhysicalAddress, ref TValue recordValue) - { - // Called after a new record is allocated - if (RevivificationManager.IsFixedLength) - return (RevivificationManager.FixedValueLength, RevivificationManager.FixedValueLength); - - int valueOffset = GetValueOffset(newPhysicalAddress, ref recordValue); - int usedValueLength = actualSize - valueOffset; - int fullValueLength = allocatedSize - valueOffset; - Debug.Assert(usedValueLength >= 0, $"GetNewValueLengths: usedValueLength {usedValueLength}"); - Debug.Assert(fullValueLength >= 0, $"GetNewValueLengths: fullValueLength {fullValueLength}"); - Debug.Assert(fullValueLength >= RoundUp(usedValueLength, sizeof(int)), $"GetNewValueLengths: usedValueLength {usedValueLength} cannot be > fullValueLength {fullValueLength}"); - - return (usedValueLength, fullValueLength); - } - - // A "free record" is one on the FreeList. - #region FreeRecords - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SetFreeRecordSize(long physicalAddress, ref RecordInfo recordInfo, int allocatedSize) - { - // Skip the valuelength calls if we are not VarLen. - if (RevivificationManager.IsFixedLength) - { - recordInfo.ClearHasFiller(); - return; - } - - // Store the full value length. Defer clearing the Key until the record is revivified (it may never be). - ref TValue recordValue = ref hlog.GetValue(physicalAddress); - int usedValueLength = hlog.GetValueLength(ref recordValue); - int fullValueLength = allocatedSize - GetValueOffset(physicalAddress, ref recordValue); - SetVarLenExtraValueLength(ref recordValue, ref recordInfo, usedValueLength, fullValueLength); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal int GetFreeRecordSize(long physicalAddress, ref RecordInfo recordInfo) - => RevivificationManager.IsFixedLength - ? hlog.GetAverageRecordSize() - : GetRecordLengths(physicalAddress, ref hlog.GetValue(physicalAddress), ref recordInfo).fullRecordLength; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - static void ClearExtraValueSpace(ref RecordInfo recordInfo, ref TValue recordValue, int usedValueLength, int fullValueLength) - { - // SpanByte's implementation of GetAndInitializeValue does not clear the space after usedValueLength. This may be - // considerably less than the previous value length, so we clear it here before DisposeForRevivification. This space - // includes the extra value length if Filler is set, so we must clear the space before clearing the Filler bit so - // log-scan traversal does not see nonzero values past Value (it's fine if we see the Filler and extra length is 0). - int extraValueLength = fullValueLength - usedValueLength; // do not round up usedValueLength; we must clear all extra bytes - if (extraValueLength > 0) - { - // Even though this says "SpanByte" it is just a utility function to zero space; no actual SpanByte instance is assumed - SpanByte.Clear((byte*)Unsafe.AsPointer(ref recordValue) + usedValueLength, extraValueLength); - } - recordInfo.ClearHasFiller(); - } - - // Do not try to inline this; it causes TryAllocateRecord to bloat and slow - bool TryTakeFreeRecord(TSessionFunctionsWrapper sessionFunctions, int requiredSize, ref int allocatedSize, int newKeySize, long minRevivAddress, - out long logicalAddress, out long physicalAddress) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - { - // Caller checks for UseFreeRecordPool - if (RevivificationManager.TryTake(allocatedSize, minRevivAddress, out logicalAddress, ref sessionFunctions.Ctx.RevivificationStats)) - { - physicalAddress = hlog.GetPhysicalAddress(logicalAddress); - ref RecordInfo recordInfo = ref hlog.GetInfo(physicalAddress); - Debug.Assert(recordInfo.IsSealed, "TryTakeFreeRecord: recordInfo should still have the revivification Seal"); - - // If IsFixedLengthReviv, the allocatedSize will be unchanged - if (!RevivificationManager.IsFixedLength) - { - var (usedValueLength, fullValueLength, fullRecordLength) = GetRecordLengths(physicalAddress, ref hlog.GetValue(physicalAddress), ref recordInfo); - - // ClearExtraValueSpace has already been called (at freelist-add time) to zero the end of the value space between used and full value lengths and clear the Filler. - // Now we use the newKeySize to find out how much space is actually required. - var valueOffset = fullRecordLength - fullValueLength; - var requiredValueLength = requiredSize - valueOffset; - var minValueLength = requiredValueLength < usedValueLength ? requiredValueLength : usedValueLength; - ref var recordValue = ref hlog.GetValue(physicalAddress); - Debug.Assert(valueOffset == (long)Unsafe.AsPointer(ref recordValue) - physicalAddress); - - // Clear any no-longer-needed space, then call DisposeForRevivification again with newKeySize so SpanByte can be efficient about zeroinit. - ClearExtraValueSpace(ref recordInfo, ref recordValue, minValueLength, fullValueLength); - storeFunctions.DisposeRecord(ref hlog.GetKey(physicalAddress), ref recordValue, DisposeReason.RevivificationFreeList, newKeySize); - - Debug.Assert(fullRecordLength >= allocatedSize, $"TryTakeFreeRecord: fullRecordLength {fullRecordLength} should be >= allocatedSize {allocatedSize}"); - allocatedSize = fullRecordLength; - } - - // Preserve the Sealed bit due to checkpoint/recovery; see RecordInfo.WriteInfo. - return true; - } - - // No free record available. - logicalAddress = physicalAddress = default; - return false; - } - - #endregion FreeRecords - - // TombstonedRecords are in the tag chain with the tombstone bit set (they are not in the freelist). They preserve the key (they mark that key as deleted, - // which is important if there is a subsequent record for that key), and store the full Value length after the used value data (if there is room). - #region TombstonedRecords - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SetTombstoneAndExtraValueLength(ref TValue recordValue, ref RecordInfo recordInfo, int usedValueLength, int fullValueLength) - { - recordInfo.SetTombstone(); - if (RevivificationManager.IsFixedLength) - { - recordInfo.ClearHasFiller(); - return; - } - - Debug.Assert(usedValueLength == hlog.GetValueLength(ref recordValue)); - SetVarLenExtraValueLength(ref recordValue, ref recordInfo, usedValueLength, fullValueLength); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal (bool ok, int usedValueLength) TryReinitializeTombstonedValue(TSessionFunctionsWrapper sessionFunctions, - ref RecordInfo srcRecordInfo, ref TKey key, ref TValue recordValue, int requiredSize, (int usedValueLength, int fullValueLength, int allocatedSize) recordLengths) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper - { - if (RevivificationManager.IsFixedLength || recordLengths.allocatedSize < requiredSize) - return (false, recordLengths.usedValueLength); - - // Zero the end of the value space between required and full value lengths and clear the Filler. - var valueOffset = recordLengths.allocatedSize - recordLengths.fullValueLength; - var requiredValueLength = requiredSize - valueOffset; - var minValueLength = requiredValueLength < recordLengths.usedValueLength ? requiredValueLength : recordLengths.usedValueLength; - - ClearExtraValueSpace(ref srcRecordInfo, ref recordValue, minValueLength, recordLengths.fullValueLength); - storeFunctions.DisposeRecord(ref key, ref recordValue, DisposeReason.RevivificationFreeList); - - srcRecordInfo.ClearTombstone(); - - SetExtraValueLength(ref recordValue, ref srcRecordInfo, recordLengths.usedValueLength, recordLengths.fullValueLength); - return (true, hlog.GetValueLength(ref recordValue)); - } - - #endregion TombstonedRecords - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationManager.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationManager.cs index ee794a8045e..d49da7174fc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationManager.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationManager.cs @@ -5,27 +5,23 @@ namespace Tsavorite.core { - internal struct RevivificationManager - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal struct RevivificationManager + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - internal FreeRecordPool FreeRecordPool; + internal FreeRecordPool FreeRecordPool; internal readonly bool UseFreeRecordPool => FreeRecordPool is not null; internal RevivificationStats stats = new(); internal readonly bool IsEnabled = false; - internal static int FixedValueLength => Unsafe.SizeOf(); internal bool restoreDeletedRecordsIfBinIsFull; internal bool useFreeRecordPoolForCTT; - internal readonly bool IsFixedLength { get; } - internal double revivifiableFraction; - public RevivificationManager(TsavoriteKV store, bool isFixedLen, RevivificationSettings revivSettings, LogSettings logSettings) + public RevivificationManager(TsavoriteKV store, RevivificationSettings revivSettings, LogSettings logSettings) { - IsFixedLength = isFixedLen; revivifiableFraction = revivSettings is null || revivSettings.RevivifiableFraction == RevivificationSettings.DefaultRevivifiableFraction ? logSettings.MutableFraction : revivSettings.RevivifiableFraction; @@ -33,36 +29,32 @@ public RevivificationManager(TsavoriteKV 0) { - FreeRecordPool = new FreeRecordPool(store, revivSettings, IsFixedLength ? store.hlog.GetAverageRecordSize() : -1); + FreeRecordPool = new FreeRecordPool(store, revivSettings); restoreDeletedRecordsIfBinIsFull = revivSettings.RestoreDeletedRecordsIfBinIsFull; useFreeRecordPoolForCTT = revivSettings.UseFreeRecordPoolForCopyToTail; } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal long GetMinRevivifiableAddress(long tailAddress, long readOnlyAddress) + internal readonly long GetMinRevivifiableAddress(long tailAddress, long readOnlyAddress) => tailAddress - (long)((tailAddress - readOnlyAddress) * revivifiableFraction); // Method redirectors [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryAdd(long logicalAddress, int size, ref RevivificationStats revivStats) - => UseFreeRecordPool && FreeRecordPool.TryAdd(logicalAddress, size, ref revivStats); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryAdd(long logicalAddress, long physicalAddress, int allocatedSize, ref RevivificationStats revivStats) - => UseFreeRecordPool && FreeRecordPool.TryAdd(logicalAddress, physicalAddress, allocatedSize, ref revivStats); + public bool TryAdd(long logicalAddress, ref LogRecord logRecord, ref RevivificationStats revivStats) + => UseFreeRecordPool && FreeRecordPool.TryAdd(logicalAddress, ref logRecord, ref revivStats); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryTake(int recordSize, long minAddress, out long address, ref RevivificationStats revivStats) + public bool TryTake(ref RecordSizeInfo sizeInfo, long minAddress, out long address, ref RevivificationStats revivStats) { if (UseFreeRecordPool) - return FreeRecordPool.TryTake(recordSize, minAddress, out address, ref revivStats); + return FreeRecordPool.TryTake(ref sizeInfo, minAddress, out address, ref revivStats); address = 0; return false; } @@ -70,9 +62,7 @@ public bool TryTake(int recordSize, long minAddress, out long address, ref Reviv public void Dispose() { if (UseFreeRecordPool) - { FreeRecordPool.Dispose(); - } } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationSettings.cs index 632036026b1..75cb541d7a5 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Revivification/RevivificationSettings.cs @@ -19,12 +19,12 @@ public class RevivificationSettings /// /// Indicates whether deleted record space should be reused. /// - ///
  • If this is true, then tombstoned records in the hashtable chain are revivified if possible, and a FreeList is maintained if + /// If this is true, then tombstoned records in the hashtable chain are revivified if possible, and a FreeList is maintained if /// is non-null and non-empty. - ///
  • - ///
  • If this is false, then tombstoned records in the hashtable chain will not be revivified, and no FreeList is used (regardless + /// + /// If this is false, then tombstoned records in the hashtable chain will not be revivified, and no FreeList is used (regardless /// of the setting of ). - ///
  • + /// ///
    ///
    public bool EnableRevivification = true; @@ -40,11 +40,6 @@ public class RevivificationSettings /// /// Bin definitions for the free list (in addition to any in the hash chains). These must be ordered by . /// - /// - /// If the Key and Value are both fixed-length datatypes (either blittable or object), this must contain a single bin whose - /// is ignored. Otherwise, one or both of the Key and Value are variable-length, - /// and this usually contains multiple bins. - /// public RevivificationBin[] FreeRecordBins; /// @@ -73,21 +68,6 @@ public class RevivificationSettings /// public static PowerOf2BinsRevivificationSettings PowerOf2Bins { get; } = new(); - /// - /// Default bin for fixed-length. - /// - public static RevivificationSettings DefaultFixedLength { get; } = new() - { - FreeRecordBins = - [ - new RevivificationBin() - { - RecordSize = RevivificationBin.MaxRecordSize, - BestFitScanLimit = RevivificationBin.UseFirstFit - } - ] - }; - /// /// Enable only in-tag-chain revivification; do not use FreeList /// @@ -98,12 +78,10 @@ public class RevivificationSettings ///
    public static RevivificationSettings None { get; } = new() { EnableRevivification = false }; - internal void Verify(bool isFixedRecordLength, double mutableFraction) + internal void Verify(double mutableFraction) { if (!EnableRevivification || FreeRecordBins?.Length == 0) return; - if (isFixedRecordLength && FreeRecordBins?.Length > 1) - throw new TsavoriteException($"Only 1 bin may be specified with fixed-length datatypes (blittable or object)"); if (RevivifiableFraction != DefaultRevivifiableFraction) { if (RevivifiableFraction <= 0) @@ -114,7 +92,7 @@ internal void Verify(bool isFixedRecordLength, double mutableFraction) if (FreeRecordBins is not null) { foreach (var bin in FreeRecordBins) - bin.Verify(isFixedRecordLength); + bin.Verify(); } } @@ -188,21 +166,17 @@ public struct RevivificationBin public const int DefaultRecordsPerBin = 256; /// - /// The maximum size of records in this partition. This should be partitioned for your app. Ignored if this is the single bin - /// for fixed-length records. + /// The maximum size of records in this partition. This should be partitioned for your app. /// public int RecordSize; /// /// The number of records for each partition. This count will be adjusted upward so the partition is cache-line aligned. /// - /// - /// The first record is not available; its space is used to store the circular buffer read and write pointers - /// public int NumberOfRecords = DefaultRecordsPerBin; /// - /// The maximum number of entries to scan for best fit after finding first fit. Ignored for fixed-length datatypes. + /// The maximum number of entries to scan for best fit after finding first fit. /// public int BestFitScanLimit = UseFirstFit; @@ -213,10 +187,8 @@ public RevivificationBin() { } - internal void Verify(bool isFixedLength) + internal void Verify() { - if (!isFixedLength && (RecordSize < MinRecordSize || RecordSize > MaxRecordSize)) - throw new TsavoriteException($"Invalid RecordSize {RecordSize}; must be >= {MinRecordSize} and <= {MaxRecordSize}"); if (NumberOfRecords < MinRecordsPerBin) throw new TsavoriteException($"Invalid NumberOfRecords {NumberOfRecords}; must be > {MinRecordsPerBin}"); if (BestFitScanLimit < 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 d8b6986fbf0..7bf580f7402 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 @@ -6,9 +6,9 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal void SplitAllBuckets() { @@ -118,23 +118,22 @@ private void SplitChunk( { entry.word = *(((long*)src_start) + index); if (Constants.kInvalidEntry == entry.word) - { continue; - } var logicalAddress = entry.Address; long physicalAddress = 0; + LogRecord logRecord = default; if (entry.ReadCache && entry.AbsoluteAddress >= readCacheBase.HeadAddress) - physicalAddress = readcache.GetPhysicalAddress(entry.AbsoluteAddress); + logRecord = readcache.CreateLogRecord(entry.AbsoluteAddress); else if (logicalAddress >= hlogBase.HeadAddress) - physicalAddress = hlog.GetPhysicalAddress(logicalAddress); + logRecord = hlog.CreateLogRecord(logicalAddress); // It is safe to always use hlog instead of readcache for some calls such // as GetKey and GetInfo - if (physicalAddress != 0) + if (logRecord.IsSet) { - var hash = storeFunctions.GetKeyHashCode64(ref hlog.GetKey(physicalAddress)); + var hash = storeFunctions.GetKeyHashCode64(logRecord.Key); if ((hash & state[resizeInfo.version].size_mask) >> (state[resizeInfo.version].size_bits - 1) == 0) { // Insert in left @@ -151,7 +150,7 @@ private void SplitChunk( left++; // Insert previous address in right - entry.Address = TraceBackForOtherChainStart(hlog.GetInfo(physicalAddress).PreviousAddress, 1); + entry.Address = TraceBackForOtherChainStart(LogRecord.GetInfo(physicalAddress).PreviousAddress, 1); if ((entry.Address != Constants.kInvalidAddress) && (entry.Address != Constants.kTempInvalidAddress)) { if (right == right_end) @@ -183,7 +182,7 @@ private void SplitChunk( right++; // Insert previous address in left - entry.Address = TraceBackForOtherChainStart(hlog.GetInfo(physicalAddress).PreviousAddress, 0); + entry.Address = TraceBackForOtherChainStart(LogRecord.GetInfo(physicalAddress).PreviousAddress, 0); if ((entry.Address != Constants.kInvalidAddress) && (entry.Address != Constants.kTempInvalidAddress)) { if (left == left_end) @@ -242,32 +241,25 @@ private long TraceBackForOtherChainStart(long logicalAddress, int bit) { while (true) { - HashBucketEntry entry = default; - entry.Address = logicalAddress; + HashBucketEntry entry = new() { Address = logicalAddress }; + LogRecord logRecord; if (entry.ReadCache) { if (logicalAddress < readCacheBase.HeadAddress) break; - var physicalAddress = readcache.GetPhysicalAddress(logicalAddress); - var hash = storeFunctions.GetKeyHashCode64(ref readcache.GetKey(physicalAddress)); - if ((hash & state[resizeInfo.version].size_mask) >> (state[resizeInfo.version].size_bits - 1) == bit) - { - return logicalAddress; - } - logicalAddress = readcache.GetInfo(physicalAddress).PreviousAddress; + logRecord = new LogRecord(readcache.GetPhysicalAddress(logicalAddress)); } else { if (logicalAddress < hlogBase.HeadAddress) break; - var physicalAddress = hlog.GetPhysicalAddress(logicalAddress); - var hash = storeFunctions.GetKeyHashCode64(ref hlog.GetKey(physicalAddress)); - if ((hash & state[resizeInfo.version].size_mask) >> (state[resizeInfo.version].size_bits - 1) == bit) - { - return logicalAddress; - } - logicalAddress = hlog.GetInfo(physicalAddress).PreviousAddress; + logRecord = new LogRecord(hlog.GetPhysicalAddress(logicalAddress)); } + + var hash = storeFunctions.GetKeyHashCode64(logRecord.Key); + if ((hash & state[resizeInfo.version].size_mask) >> (state[resizeInfo.version].size_bits - 1) == bit) + return logicalAddress; + logicalAddress = logRecord.Info.PreviousAddress; } return logicalAddress; } 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 a1209777217..da163267057 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,60 +3,44 @@ namespace Tsavorite.core { - public unsafe partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Copy a record from the disk to the read cache. /// /// - /// - /// - /// - /// Contains the and structures for this operation, + /// Input log record that was IO'd from disk + /// Contains the and structures for this operation, /// and allows passing back the newLogicalAddress for invalidation in the case of exceptions. /// /// True if copied to readcache, else false; readcache is "best effort", and we don't fail the read process, or slow it down by retrying. /// - internal bool TryCopyToReadCache(TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext, - ref TKey key, ref TInput input, ref TValue recordValue, ref OperationStackContext stackCtx) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal bool TryCopyToReadCache(ref TSourceLogRecord inputLogRecord, TSessionFunctionsWrapper sessionFunctions, + ref PendingContext pendingContext, ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - var (actualSize, allocatedSize, _) = hlog.GetRecordSize(ref key, ref recordValue); + var sizeInfo = new RecordSizeInfo() { FieldInfo = inputLogRecord.GetRecordFieldInfo() }; + hlog.PopulateRecordSizeInfo(ref sizeInfo); - if (!TryAllocateRecordReadCache(ref pendingContext, ref stackCtx, allocatedSize, out long newLogicalAddress, out long newPhysicalAddress, out _)) + if (!TryAllocateRecordReadCache(ref pendingContext, ref stackCtx, ref sizeInfo, out var newLogicalAddress, out var newPhysicalAddress, out var allocatedSize, out _)) return false; - ref var newRecordInfo = ref WriteNewRecordInfo(ref key, readCacheBase, newPhysicalAddress, inNewVersion: false, stackCtx.hei.Address); + var newLogRecord = WriteNewRecordInfo(inputLogRecord.Key, readCacheBase, newLogicalAddress, newPhysicalAddress, inNewVersion: false, previousAddress: stackCtx.hei.Address); stackCtx.SetNewRecord(newLogicalAddress | Constants.kReadCacheBitMask); - UpsertInfo upsertInfo = new() - { - Version = sessionFunctions.Ctx.version, - SessionID = sessionFunctions.Ctx.sessionID, - Address = Constants.kInvalidAddress, // We do not expose readcache addresses - KeyHash = stackCtx.hei.hash, - }; - upsertInfo.SetRecordInfo(ref newRecordInfo); - // Even though readcache records are immutable, we have to initialize the lengths - ref TValue newRecordValue = ref readcache.GetAndInitializeValue(newPhysicalAddress, newPhysicalAddress + actualSize); - (upsertInfo.UsedValueLength, upsertInfo.FullValueLength) = GetNewValueLengths(actualSize, allocatedSize, newPhysicalAddress, ref newRecordValue); - - TOutput output = default; - if (!sessionFunctions.SingleWriter(ref key, ref input, ref recordValue, ref readcache.GetAndInitializeValue(newPhysicalAddress, newPhysicalAddress + actualSize), - ref output, ref upsertInfo, WriteReason.CopyToReadCache, ref newRecordInfo)) - { - stackCtx.SetNewRecordInvalid(ref newRecordInfo); - return false; - } + readcache.InitializeValue(newPhysicalAddress, ref sizeInfo); + newLogRecord.SetFillerLength(allocatedSize); + newLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); // Insert the new record by CAS'ing directly into the hash entry (readcache records are always CAS'd into the HashBucketEntry, never spliced). // It is possible that we will successfully CAS but subsequently fail due to a main log entry having been spliced in. var success = stackCtx.hei.TryCAS(newLogicalAddress | Constants.kReadCacheBitMask); var casSuccess = success; - OperationStatus failStatus = OperationStatus.RETRY_NOW; // Default to CAS-failed status, which does not require an epoch refresh + var failStatus = OperationStatus.RETRY_NOW; // Default to CAS-failed status, which does not require an epoch refresh if (success && stackCtx.recSrc.LowestReadCacheLogicalAddress != Constants.kInvalidAddress) { // If someone added a main-log entry for this key from a CTT while we were inserting the new readcache record, then the new @@ -68,26 +52,24 @@ internal bool TryCopyToReadCache : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public unsafe partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Copy a record from the immutable region of the log, from the disk, or from ConditionalCopyToTail to the tail of the log (or splice into the log/readcache boundary). /// + /// The source record, either from readonly region of the in-memory log, or from disk + /// /// - /// - /// - /// - /// - /// Contains the and structures for this operation, + /// Contains the and structures for this operation, /// and allows passing back the newLogicalAddress for invalidation in the case of exceptions. - /// if ., the recordInfo to close, if transferring. - /// - /// The reason for this operation. /// /// /// RETRY_NOW: failed CAS, so no copy done. This routine deals entirely with new records, so will not encounter Sealed records /// SUCCESS: copy was done /// /// - internal OperationStatus TryCopyToTail(ref PendingContext pendingContext, - ref TKey key, ref TInput input, ref TValue value, ref TOutput output, ref OperationStackContext stackCtx, - ref RecordInfo srcRecordInfo, TSessionFunctionsWrapper sessionFunctions, WriteReason reason) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal OperationStatus TryCopyToTail(ref TSourceLogRecord inputLogRecord, + TSessionFunctionsWrapper sessionFunctions, ref PendingContext pendingContext, + ref OperationStackContext stackCtx) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { - var (actualSize, allocatedSize, keySize) = hlog.GetRecordSize(ref key, ref value); - if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, actualSize, ref allocatedSize, keySize, new AllocateOptions() { Recycle = true }, - out long newLogicalAddress, out long newPhysicalAddress, out OperationStatus status)) + var sizeInfo = new RecordSizeInfo() { FieldInfo = inputLogRecord.GetRecordFieldInfo() }; + hlog.PopulateRecordSizeInfo(ref sizeInfo); + + var allocOptions = new AllocateOptions() { recycle = true }; + if (!TryAllocateRecord(sessionFunctions, ref pendingContext, ref stackCtx, ref sizeInfo, allocOptions, out var newLogicalAddress, out var newPhysicalAddress, out var allocatedSize, out var status)) return status; - ref var newRecordInfo = ref WriteNewRecordInfo(ref key, hlogBase, newPhysicalAddress, inNewVersion: sessionFunctions.Ctx.InNewVersion, stackCtx.recSrc.LatestLogicalAddress); + var newLogRecord = WriteNewRecordInfo(inputLogRecord.Key, hlogBase, newLogicalAddress, newPhysicalAddress, inNewVersion: sessionFunctions.Ctx.InNewVersion, previousAddress: stackCtx.recSrc.LatestLogicalAddress); stackCtx.SetNewRecord(newLogicalAddress); - UpsertInfo upsertInfo = new() - { - Version = sessionFunctions.Ctx.version, - SessionID = sessionFunctions.Ctx.sessionID, - Address = newLogicalAddress, - KeyHash = stackCtx.hei.hash, - }; - upsertInfo.SetRecordInfo(ref newRecordInfo); - - ref TValue newRecordValue = ref hlog.GetAndInitializeValue(newPhysicalAddress, newPhysicalAddress + actualSize); - (upsertInfo.UsedValueLength, upsertInfo.FullValueLength) = GetNewValueLengths(actualSize, allocatedSize, newPhysicalAddress, ref newRecordValue); - - if (!sessionFunctions.SingleWriter(ref key, ref input, ref value, ref newRecordValue, ref output, ref upsertInfo, reason, ref newRecordInfo)) - { - // Save allocation for revivification (not retry, because we won't retry here), or abandon it if that fails. - if (RevivificationManager.UseFreeRecordPool && RevivificationManager.TryAdd(newLogicalAddress, newPhysicalAddress, allocatedSize, ref sessionFunctions.Ctx.RevivificationStats)) - stackCtx.ClearNewRecord(); - else - stackCtx.SetNewRecordInvalid(ref newRecordInfo); - return (upsertInfo.Action == UpsertAction.CancelOperation) ? OperationStatus.CANCELED : OperationStatus.SUCCESS; - } - SetExtraValueLength(ref newRecordValue, ref srcRecordInfo, upsertInfo.UsedValueLength, upsertInfo.FullValueLength); + hlog.InitializeValue(newPhysicalAddress, ref sizeInfo); + newLogRecord.SetFillerLength(allocatedSize); + newLogRecord.TryCopyFrom(ref inputLogRecord, ref sizeInfo); // Insert the new record by CAS'ing either directly into the hash entry or splicing into the readcache/mainlog boundary. - bool success = CASRecordIntoChain(ref key, ref stackCtx, newLogicalAddress, ref newRecordInfo); + var success = CASRecordIntoChain(newLogicalAddress, ref newLogRecord, ref stackCtx); if (success) { - newRecordInfo.UnsealAndValidate(); - PostCopyToTail(ref key, ref stackCtx, ref srcRecordInfo, pendingContext.InitialEntryAddress); + newLogRecord.InfoRef.UnsealAndValidate(); - pendingContext.recordInfo = newRecordInfo; - pendingContext.logicalAddress = upsertInfo.Address; - sessionFunctions.PostSingleWriter(ref key, ref input, ref value, ref hlog.GetValue(newPhysicalAddress), ref output, ref upsertInfo, reason, ref newRecordInfo); + pendingContext.logicalAddress = newLogicalAddress; stackCtx.ClearNewRecord(); return OperationStatusUtils.AdvancedOpCode(OperationStatus.SUCCESS, StatusCode.Found | StatusCode.CopiedRecord); } // CAS failed - stackCtx.SetNewRecordInvalid(ref newRecordInfo); - storeFunctions.DisposeRecord(ref hlog.GetKey(newPhysicalAddress), ref hlog.GetValue(newPhysicalAddress), DisposeReason.SingleWriterCASFailed); + stackCtx.SetNewRecordInvalid(ref newLogRecord.InfoRef); + DisposeRecord(ref newLogRecord, DisposeReason.InitialWriterCASFailed); - SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress, allocatedSize); + SaveAllocationForRetry(ref pendingContext, newLogicalAddress, newPhysicalAddress); return OperationStatus.RETRY_NOW; // CAS failure does not require epoch refresh } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/UpsertValueSelector.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/UpsertValueSelector.cs new file mode 100644 index 00000000000..565a5938e3d --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/UpsertValueSelector.cs @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Tsavorite.core +{ + public unsafe partial class TsavoriteKV where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + /// + /// Eliminates switching on type of value or another variable to determine which overloaded value-taking method to call. + /// + internal interface IUpsertValueSelector + { + static abstract RecordSizeInfo GetUpsertRecordSize(TAllocator allocator, ReadOnlySpan key, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput; + + static abstract bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper; + + static abstract void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper; + + static abstract bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper; + } + + internal struct SpanUpsertValueSelector : IUpsertValueSelector + { + public static RecordSizeInfo GetUpsertRecordSize(TAllocator allocator, ReadOnlySpan key, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => allocator.GetUpsertRecordSize(key, valueSpan, ref input, varlenInput); + + public static bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InitialWriter(ref logRecord, ref sizeInfo, ref input, valueSpan, ref output, ref upsertInfo); + + public static void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, valueSpan, ref output, ref upsertInfo); + + public static bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, valueSpan, ref output, ref upsertInfo); + } + + internal struct ObjectUpsertValueSelector : IUpsertValueSelector + { + public static RecordSizeInfo GetUpsertRecordSize(TAllocator allocator, ReadOnlySpan key, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => allocator.GetUpsertRecordSize(key, valueObject, ref input, varlenInput); + + public static bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InitialWriter(ref logRecord, ref sizeInfo, ref input, valueObject, ref output, ref upsertInfo); + + public static void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, valueObject, ref output, ref upsertInfo); + + public static bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, valueObject, ref output, ref upsertInfo); + } + + internal struct LogRecordUpsertValueSelector : IUpsertValueSelector + { + public static RecordSizeInfo GetUpsertRecordSize(TAllocator allocator, ReadOnlySpan key, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TInput input, TVariableLengthInput varlenInput) + where TSourceLogRecord : ISourceLogRecord + where TVariableLengthInput : IVariableLengthInput + => allocator.GetUpsertRecordSize(key, ref inputLogRecord, ref input, varlenInput); + + public static bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InitialWriter(ref logRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo); + + public static void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo); + + public static bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TInput input, + ReadOnlySpan valueSpan, IHeapObject valueObject, ref TSourceLogRecord inputLogRecord, ref TOutput output, ref UpsertInfo upsertInfo, TSessionFunctionsWrapper sessionFunctions) + where TSourceLogRecord : ISourceLogRecord + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + => sessionFunctions.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, ref inputLogRecord, ref output, ref upsertInfo); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/LogAccessor.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/LogAccessor.cs index a12ee914dd5..3ae9cec35f8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/LogAccessor.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/LogAccessor.cs @@ -10,20 +10,20 @@ namespace Tsavorite.core /// /// Wrapper to process log-related commands /// - public sealed class LogAccessor : IObservable> - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public sealed class LogAccessor : IObservable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - private readonly TsavoriteKV store; + private readonly TsavoriteKV store; private readonly TAllocator allocator; - private readonly AllocatorBase allocatorBase; + private readonly AllocatorBase allocatorBase; /// /// Constructor /// /// /// - internal LogAccessor(TsavoriteKV store, TAllocator allocator) + internal LogAccessor(TsavoriteKV store, TAllocator allocator) { this.store = store; this.allocator = allocator; @@ -55,14 +55,6 @@ internal LogAccessor(TsavoriteKV stor ///
    public long BeginAddress => allocatorBase.BeginAddress; - /// - /// Get the bytes used on the primary log by every record. Does not include - /// the size of variable-length inline data. Note that class objects occupy - /// 8 bytes (reference) on the main log (i.e., the heap space occupied by - /// class objects is not included in the result of this call). - /// - public int FixedRecordSize => allocator.GetFixedRecordSize(); - /// /// Number of pages left empty or unallocated in the in-memory buffer (between 0 and BufferSize-1) /// @@ -199,7 +191,7 @@ public Func IsSizeBeyondLimit /// To scan the historical part of the log, use the Scan(...) method ///
    /// Observer to which scan iterator is pushed - public IDisposable Subscribe(IObserver> readOnlyObserver) + public IDisposable Subscribe(IObserver readOnlyObserver) { allocatorBase.OnReadOnlyObserver = readOnlyObserver; return new LogSubscribeDisposable(allocatorBase, isReadOnly: true); @@ -212,13 +204,13 @@ public IDisposable Subscribe(IObserver> rea /// To scan the historical part of the log, use the Scan(...) method ///
    /// Observer to which scan iterator is pushed - public IDisposable SubscribeEvictions(IObserver> evictionObserver) + public IDisposable SubscribeEvictions(IObserver evictionObserver) { allocatorBase.OnEvictionObserver = evictionObserver; return new LogSubscribeDisposable(allocatorBase, isReadOnly: false); } - public IDisposable SubscribeDeserializations(IObserver> deserializationObserver) + public IDisposable SubscribeDeserializations(IObserver deserializationObserver) { allocatorBase.OnDeserializationObserver = deserializationObserver; return new LogSubscribeDisposable(allocatorBase, isReadOnly: false); @@ -229,10 +221,10 @@ public IDisposable SubscribeDeserializations(IObserver class LogSubscribeDisposable : IDisposable { - private readonly AllocatorBase allocator; + private readonly AllocatorBase allocator; private readonly bool readOnly; - public LogSubscribeDisposable(AllocatorBase allocator, bool isReadOnly) + public LogSubscribeDisposable(AllocatorBase allocator, bool isReadOnly) { this.allocator = allocator; readOnly = isReadOnly; @@ -285,114 +277,57 @@ public void ShiftReadOnlyAddress(long newReadOnlyAddress, bool wait) ///
    /// Scan iterator instance [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ITsavoriteScanIterator Scan(long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering, bool includeSealedRecords = false) + public ITsavoriteScanIterator Scan(long beginAddress, long endAddress, DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering, bool includeSealedRecords = false) => allocatorBase.Scan(store: null, beginAddress, endAddress, scanBufferingMode, includeSealedRecords); /// /// Push-scan the log given address range; returns all records with address less than endAddress /// /// True if Scan completed; false if Scan ended early due to one of the TScanIterator reader functions returning false - public bool Scan(ref TScanFunctions scanFunctions, long beginAddress, long endAddress, ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering) - where TScanFunctions : IScanIteratorFunctions + public bool Scan(ref TScanFunctions scanFunctions, long beginAddress, long endAddress, DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering) + where TScanFunctions : IScanIteratorFunctions => allocatorBase.Scan(store, beginAddress, endAddress, ref scanFunctions, scanBufferingMode); /// /// Iterate versions of the specified key, starting with most recent /// /// True if Scan completed; false if Scan ended early due to one of the TScanIterator reader functions returning false - public bool IterateKeyVersions(ref TScanFunctions scanFunctions, ref TKey key) - where TScanFunctions : IScanIteratorFunctions - => allocatorBase.IterateKeyVersions(store, ref key, ref scanFunctions); + public bool IterateKeyVersions(ref TScanFunctions scanFunctions, ReadOnlySpan key) + where TScanFunctions : IScanIteratorFunctions + => allocatorBase.IterateKeyVersions(store, key, ref scanFunctions); /// /// Flush log until current tail (records are still retained in memory) /// /// Synchronous wait for operation to complete - public void Flush(bool wait) - { - ShiftReadOnlyAddress(allocatorBase.GetTailAddress(), wait); - } + public void Flush(bool wait) => ShiftReadOnlyAddress(allocatorBase.GetTailAddress(), wait); /// /// Flush log and evict all records from memory /// /// Wait for operation to complete - public void FlushAndEvict(bool wait) - { - ShiftHeadAddress(allocatorBase.GetTailAddress(), wait); - } - - /// - /// Delete log entirely from memory. Cannot allocate on the log - /// after this point. This is a synchronous operation. - /// - public void DisposeFromMemory() - { - // Ensure we have flushed and evicted - FlushAndEvict(true); - - // Delete from memory - allocatorBase.DeleteFromMemory(); - } + public void FlushAndEvict(bool wait) => ShiftHeadAddress(allocatorBase.GetTailAddress(), wait); /// /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate /// - /// Functions used to manage key-values during compaction /// Compact log until this address /// Compaction type (whether we lookup records or scan log for liveness checking) /// Address until which compaction was done - public long Compact(TFunctions functions, long untilAddress, CompactionType compactionType) - where TFunctions : ISessionFunctions - => Compact>(functions, default, untilAddress, compactionType); - - /// - /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log - /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate - /// - /// Functions used to manage key-values during compaction - /// Input for SingleWriter - /// Output from SingleWriter; it will be called all records that are moved, before Compact() returns, so the user must supply buffering or process each output completely - /// Compact log until this address - /// Compaction type (whether we lookup records or scan log for liveness checking) - /// Address until which compaction was done - public long Compact(TFunctions functions, ref TInput input, ref TOutput output, long untilAddress, CompactionType compactionType) - where TFunctions : ISessionFunctions - => Compact>(functions, default, ref input, ref output, untilAddress, compactionType); - - /// - /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log - /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate - /// - /// Functions used to manage key-values during compaction - /// User provided compaction functions (see ) - /// Compact log until this address - /// Compaction type (whether we lookup records or scan log for liveness checking) - /// Address until which compaction was done - public long Compact(TFunctions functions, TCompactionFunctions cf, long untilAddress, CompactionType compactionType) - where TFunctions : ISessionFunctions - where TCompactionFunctions : ICompactionFunctions - { - TInput input = default; - TOutput output = default; - return Compact(functions, cf, ref input, ref output, untilAddress, compactionType); - } + public long Compact(long untilAddress, CompactionType compactionType) + => Compact(default, untilAddress, compactionType); /// /// Compact the log until specified address, moving active records to the tail of the log. BeginAddress is shifted, but the physical log /// is not deleted from disk. Caller is responsible for truncating the physical log on disk by taking a checkpoint or calling Log.Truncate /// - /// Functions used to manage key-values during compaction - /// User provided compaction functions (see ) - /// Input for SingleWriter - /// Output from SingleWriter; it will be called all records that are moved, before Compact() returns, so the user must supply buffering or process each output completely + /// User provided compaction functions (see ) /// Compact log until this address /// Compaction type (whether we lookup records or scan log for liveness checking) /// Address until which compaction was done - public long Compact(TFunctions functions, TCompactionFunctions cf, ref TInput input, ref TOutput output, long untilAddress, CompactionType compactionType) - where TFunctions : ISessionFunctions - where TCompactionFunctions : ICompactionFunctions - => store.Compact(functions, cf, ref input, ref output, untilAddress, compactionType); + public long Compact(TCompactionFunctions cf, long untilAddress, CompactionType compactionType) + where TCompactionFunctions : ICompactionFunctions + => store.Compact(cf, untilAddress, compactionType); } } \ No newline at end of file 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 05adb7badbd..b6bb0ec4d2c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -15,14 +15,14 @@ namespace Tsavorite.core /// /// The Tsavorite Key/Value store class /// - public partial class TsavoriteKV : TsavoriteBase, IDisposable - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase, IDisposable + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { internal readonly TAllocator hlog; - internal readonly AllocatorBase hlogBase; + internal readonly AllocatorBase hlogBase; internal readonly TAllocator readcache; - internal readonly AllocatorBase readCacheBase; + internal readonly AllocatorBase readCacheBase; internal readonly TStoreFunctions storeFunctions; @@ -57,20 +57,20 @@ public partial class TsavoriteKV : Ts /// /// Hybrid log used by this Tsavorite instance /// - public LogAccessor Log { get; } + public LogAccessor Log { get; } /// /// Read cache used by this Tsavorite instance /// - public LogAccessor ReadCache { get; } + public LogAccessor ReadCache { get; } int maxSessionID; - internal readonly OverflowBucketLockTable LockTable; + internal readonly OverflowBucketLockTable LockTable; internal readonly int ThrottleCheckpointFlushDelayMs = -1; - internal RevivificationManager RevivificationManager; + internal RevivificationManager RevivificationManager; internal Func allocatorFactory; @@ -80,7 +80,7 @@ public partial class TsavoriteKV : Ts /// Config settings /// Store-level user function implementations /// Func to call to create the allocator(s, if doing readcache) - public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFunctions, Func allocatorFactory) + public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFunctions, Func allocatorFactory) : base(kvSettings.Epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger("TsavoriteKV Index Overflow buckets")) { this.allocatorFactory = allocatorFactory; @@ -118,8 +118,6 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun if (ReadCopyOptions.CopyFrom == ReadCopyFrom.Inherit) ReadCopyOptions.CopyFrom = ReadCopyFrom.Device; - bool isFixedLenReviv = hlog.IsFixedLength; - // Create the allocator var allocatorSettings = new AllocatorSettings(logSettings, epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger(typeof(TAllocator).Name)); hlog = allocatorFactory(allocatorSettings, storeFunctions); @@ -132,7 +130,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun allocatorSettings.LogSettings = new() { LogDevice = new NullDevice(), - ObjectLogDevice = hlog.HasObjectLog ? new NullDevice() : null, + ObjectLogDevice = null, // TODO remove hlog.HasObjectLog ? new NullDevice() : null, PageSizeBits = logSettings.ReadCacheSettings.PageSizeBits, MemorySizeBits = logSettings.ReadCacheSettings.MemorySizeBits, SegmentSizeBits = logSettings.ReadCacheSettings.MemorySizeBits, @@ -149,8 +147,8 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun sectorSize = (int)logSettings.LogDevice.SectorSize; Initialize(kvSettings.GetIndexSizeCacheLines(), sectorSize); - LockTable = new OverflowBucketLockTable(this); - RevivificationManager = new(this, isFixedLenReviv, kvSettings.RevivificationSettings, logSettings); + LockTable = new OverflowBucketLockTable(this); + RevivificationManager = new(this, kvSettings.RevivificationSettings, logSettings); stateMachineDriver = kvSettings.StateMachineDriver ?? new(epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); @@ -165,10 +163,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun } /// Get the hashcode for a key. - public long GetKeyHash(TKey key) => storeFunctions.GetKeyHashCode64(ref key); - - /// Get the hashcode for a key. - public long GetKeyHash(ref TKey key) => storeFunctions.GetKeyHashCode64(ref key); + public long GetKeyHash(ReadOnlySpan key) => storeFunctions.GetKeyHashCode64(key); /// /// Initiate full checkpoint @@ -181,7 +176,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun /// 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) { IStateMachine stateMachine; @@ -214,7 +209,7 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// Await task to complete checkpoint, if initiated successfully /// public async ValueTask<(bool success, Guid token)> TakeFullCheckpointAsync(CheckpointType checkpointType, - CancellationToken cancellationToken = default, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + CancellationToken cancellationToken = default, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { var success = TryInitiateFullCheckpoint(out Guid token, checkpointType, streamingSnapshotIteratorFunctions); @@ -265,7 +260,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) { IStateMachine stateMachine; @@ -283,7 +278,7 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp && checkpointType == CheckpointType.Snapshot && token != default && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress - && !hlog.HasObjectLog; + ; // TODO remove && !hlog.HasObjectLog; if (incremental) { stateMachine = Checkpoint.IncrementalHybridLogOnly(this, token); @@ -308,7 +303,7 @@ public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid checkpointType == CheckpointType.Snapshot && guid != default && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress - && !hlog.HasObjectLog; + ; // TODO remove: && !hlog.HasObjectLog; } /// @@ -459,134 +454,150 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextRead(ref TKey key, ref TInput input, ref TOutput output, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ContextRead(ReadOnlySpan key, ref TInput input, ref TOutput output, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions); OperationStatus internalStatus; - var keyHash = storeFunctions.GetKeyHashCode64(ref key); + var keyHash = storeFunctions.GetKeyHashCode64(key); do - internalStatus = InternalRead(ref key, keyHash, ref input, ref output, context, ref pcontext, sessionFunctions); + internalStatus = InternalRead(key, keyHash, ref input, ref output, context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - - return status; + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextRead(ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, + internal Status ContextRead(ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions, ref readOptions); OperationStatus internalStatus; - var keyHash = readOptions.KeyHash ?? storeFunctions.GetKeyHashCode64(ref key); + var keyHash = readOptions.KeyHash ?? storeFunctions.GetKeyHashCode64(key); do - internalStatus = InternalRead(ref key, keyHash, ref input, ref output, context, ref pcontext, sessionFunctions); + internalStatus = InternalRead(key, keyHash, ref input, ref output, context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - recordMetadata = status.IsCompletedSuccessfully ? new(pcontext.recordInfo, pcontext.logicalAddress) : default; - return status; + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] internal Status ContextReadAtAddress(long address, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions, ref readOptions, noKey: true); - TKey key = default; - return ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, context, ref pcontext, sessionFunctions); + var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions, ref readOptions); + pcontext.SetIsNoKey(); + return ContextReadAtAddress(address, key: default, ref input, ref output, ref readOptions, out recordMetadata, context, ref pcontext, sessionFunctions); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ContextReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { - var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions, ref readOptions, noKey: false); - return ContextReadAtAddress(address, ref key, ref input, ref output, ref readOptions, out recordMetadata, context, ref pcontext, sessionFunctions); + var pcontext = new PendingContext(sessionFunctions.Ctx.ReadCopyOptions, ref readOptions); + return ContextReadAtAddress(address, key, ref input, ref output, ref readOptions, out recordMetadata, context, ref pcontext, sessionFunctions); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private Status ContextReadAtAddress(long address, ref TKey key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, + private Status ContextReadAtAddress(long address, ReadOnlySpan key, ref TInput input, ref TOutput output, ref ReadOptions readOptions, out RecordMetadata recordMetadata, TContext context, ref PendingContext pcontext, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + { + OperationStatus internalStatus; + do + internalStatus = InternalReadAtAddress(address, key, ref input, ref output, ref readOptions, context, ref pcontext, sessionFunctions); + while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); + + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal Status ContextUpsert(ReadOnlySpan key, long keyHash, ref TInput input, + ReadOnlySpan srcStringValue, ref TOutput output, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { + var pcontext = default(PendingContext); OperationStatus internalStatus; + DiskLogRecord emptyLogRecord = default; + do - internalStatus = InternalReadAtAddress(address, ref key, ref input, ref output, ref readOptions, context, ref pcontext, sessionFunctions); + internalStatus = InternalUpsert( + key, keyHash, ref input, srcStringValue, srcObjectValue:null, ref emptyLogRecord, ref output, ref context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - recordMetadata = status.IsCompletedSuccessfully ? new(pcontext.recordInfo, pcontext.logicalAddress) : default; - return status; + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextUpsert(ref TKey key, long keyHash, ref TInput input, ref TValue value, ref TOutput output, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ContextUpsert(ReadOnlySpan key, long keyHash, ref TInput input, + IHeapObject srcObjectValue, ref TOutput output, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var pcontext = default(PendingContext); OperationStatus internalStatus; + DiskLogRecord emptyLogRecord = default; do - internalStatus = InternalUpsert(ref key, keyHash, ref input, ref value, ref output, ref context, ref pcontext, sessionFunctions); + internalStatus = InternalUpsert( + key, keyHash, ref input, srcStringValue: default, srcObjectValue, ref emptyLogRecord, ref output, ref context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - return status; + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextUpsert(ref TKey key, long keyHash, ref TInput input, ref TValue value, ref TOutput output, out RecordMetadata recordMetadata, - TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ContextUpsert(ReadOnlySpan key, long keyHash, ref TInput input, + ref TSourceLogRecord inputLogRecord, ref TOutput output, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSourceLogRecord : ISourceLogRecord { var pcontext = default(PendingContext); OperationStatus internalStatus; do - internalStatus = InternalUpsert(ref key, keyHash, ref input, ref value, ref output, ref context, ref pcontext, sessionFunctions); + internalStatus = InternalUpsert( + key, keyHash, ref input, srcStringValue: default, srcObjectValue: default, ref inputLogRecord, ref output, ref context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - recordMetadata = status.IsCompletedSuccessfully ? new(pcontext.recordInfo, pcontext.logicalAddress) : default; - return status; + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextRMW(ref TKey key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, + internal Status ContextRMW(ReadOnlySpan key, long keyHash, ref TInput input, ref TOutput output, out RecordMetadata recordMetadata, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var pcontext = default(PendingContext); OperationStatus internalStatus; do - internalStatus = InternalRMW(ref key, keyHash, ref input, ref output, ref context, ref pcontext, sessionFunctions); + internalStatus = InternalRMW(key, keyHash, ref input, ref output, ref context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - recordMetadata = status.IsCompletedSuccessfully ? new(pcontext.recordInfo, pcontext.logicalAddress) : default; - return status; + recordMetadata = new(pcontext.logicalAddress); + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextDelete(ref TKey key, long keyHash, TContext context, TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal Status ContextDelete(ReadOnlySpan key, long keyHash, TContext context, TSessionFunctionsWrapper sessionFunctions) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { var pcontext = default(PendingContext); OperationStatus internalStatus; do - internalStatus = InternalDelete(ref key, keyHash, ref context, ref pcontext, sessionFunctions); + internalStatus = InternalDelete(key, keyHash, ref context, ref pcontext, sessionFunctions); while (HandleImmediateRetryStatus(internalStatus, sessionFunctions, ref pcontext)); - var status = HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); - return status; + return HandleOperationStatus(sessionFunctions.Ctx, ref pcontext, internalStatus); } /// @@ -598,7 +609,7 @@ public async Task GrowIndexAsync() if (epoch.ThisInstanceProtected()) throw new TsavoriteException("Cannot use GrowIndex when using non-async sessions"); - var indexResizeTask = new IndexResizeSMTask(this); + var indexResizeTask = new IndexResizeSMTask(this); var indexResizeSM = new IndexResizeSM(indexResizeTask); return await stateMachineDriver.RunAsync(indexResizeSM); } 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 640575d0957..9768fd4ada6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteIterator.cs @@ -7,9 +7,9 @@ namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { /// /// Pull iterator for all (distinct) live key-values stored in Tsavorite @@ -17,12 +17,12 @@ public partial class TsavoriteKV : Ts /// Functions used to manage key-values during iteration /// Report records until this address (tail by default) /// Tsavorite iterator - public ITsavoriteScanIterator Iterate(TFunctions functions, long untilAddress = -1) - where TFunctions : ISessionFunctions + public ITsavoriteScanIterator Iterate(TFunctions functions, long untilAddress = -1) + where TFunctions : ISessionFunctions { if (untilAddress == -1) untilAddress = Log.TailAddress; - return new TsavoriteKVIterator(this, functions, untilAddress, loggerFactory: loggerFactory); + return new TsavoriteKVIterator(this, functions, untilAddress, loggerFactory: loggerFactory); } /// @@ -33,18 +33,18 @@ public ITsavoriteScanIterator IterateReport records until this address (tail by default) /// Tsavorite iterator public bool Iterate(TFunctions functions, ref TScanFunctions scanFunctions, long untilAddress = -1) - where TFunctions : ISessionFunctions - where TScanFunctions : IScanIteratorFunctions + where TFunctions : ISessionFunctions + where TScanFunctions : IScanIteratorFunctions { if (untilAddress == -1) untilAddress = Log.TailAddress; - using TsavoriteKVIterator iter = new(this, functions, untilAddress, loggerFactory: loggerFactory); + using TsavoriteKVIterator iter = new(this, functions, untilAddress, loggerFactory: loggerFactory); if (!scanFunctions.OnStart(iter.BeginAddress, iter.EndAddress)) return false; long numRecords = 1; - bool stop = false; + var stop = false; for (; !stop && iter.PushNext(ref scanFunctions, numRecords, out stop); ++numRecords) ; @@ -53,18 +53,17 @@ public bool Iterate(TFunc } } - internal sealed class TsavoriteKVIterator : ITsavoriteScanIterator - where TFunctions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class TsavoriteKVIterator : ITsavoriteScanIterator + where TFunctions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - private readonly TsavoriteKV store; - private readonly TsavoriteKV tempKv; - private readonly ClientSession tempKvSession; - private readonly BasicContext tempbContext; - private readonly ITsavoriteScanIterator mainKvIter; - private readonly IPushScanIterator pushScanIterator; - private ITsavoriteScanIterator tempKvIter; + private readonly TsavoriteKV store; + private readonly TsavoriteKV tempKv; + private readonly ClientSession tempKvSession; + private readonly BasicContext tempbContext; + private ITsavoriteScanIterator mainKvIter; + private ITsavoriteScanIterator tempKvIter; enum IterationPhase { @@ -74,34 +73,35 @@ enum IterationPhase }; private IterationPhase iterationPhase; - public TsavoriteKVIterator(TsavoriteKV store, TFunctions functions, long untilAddress, ILoggerFactory loggerFactory = null) + public TsavoriteKVIterator(TsavoriteKV store, TFunctions functions, long untilAddress, ILoggerFactory loggerFactory = null) { this.store = store; iterationPhase = IterationPhase.MainKv; - var tempKVSettings = new KVSettings(baseDir: null, loggerFactory: loggerFactory) + var tempKVSettings = new KVSettings(baseDir: null, loggerFactory: loggerFactory) { - IndexSize = KVSettings.SetIndexSizeFromCacheLines(store.IndexSize), + IndexSize = KVSettings.SetIndexSizeFromCacheLines(store.IndexSize), LogDevice = new NullDevice(), ObjectLogDevice = new NullDevice(), MutableFraction = 1, loggerFactory = loggerFactory }; - tempKv = new TsavoriteKV(tempKVSettings, store.storeFunctions, store.allocatorFactory); + tempKv = new TsavoriteKV(tempKVSettings, store.storeFunctions, store.allocatorFactory); tempKvSession = tempKv.NewSession(functions); tempbContext = tempKvSession.BasicContext; mainKvIter = store.Log.Scan(store.Log.BeginAddress, untilAddress); - pushScanIterator = mainKvIter as IPushScanIterator; } - public long CurrentAddress => iterationPhase == IterationPhase.MainKv ? mainKvIter.CurrentAddress : tempKvIter.CurrentAddress; + ITsavoriteScanIterator CurrentIter => iterationPhase == IterationPhase.MainKv ? mainKvIter : tempKvIter; - public long NextAddress => iterationPhase == IterationPhase.MainKv ? mainKvIter.NextAddress : tempKvIter.NextAddress; + public long CurrentAddress => CurrentIter.CurrentAddress; - public long BeginAddress => iterationPhase == IterationPhase.MainKv ? mainKvIter.BeginAddress : tempKvIter.BeginAddress; + public long NextAddress => CurrentIter.NextAddress; - public long EndAddress => iterationPhase == IterationPhase.MainKv ? mainKvIter.EndAddress : tempKvIter.EndAddress; + public long BeginAddress => CurrentIter.BeginAddress; + + public long EndAddress => CurrentIter.EndAddress; public void Dispose() { @@ -111,24 +111,19 @@ public void Dispose() tempKv?.Dispose(); } - public ref TKey GetKey() => ref iterationPhase == IterationPhase.MainKv ? ref mainKvIter.GetKey() : ref tempKvIter.GetKey(); - - public ref TValue GetValue() => ref iterationPhase == IterationPhase.MainKv ? ref mainKvIter.GetValue() : ref tempKvIter.GetValue(); - - public bool GetNext(out RecordInfo recordInfo) + public bool GetNext() { while (true) { if (iterationPhase == IterationPhase.MainKv) { - if (mainKvIter.GetNext(out recordInfo)) + if (mainKvIter.GetNext()) { - ref var key = ref mainKvIter.GetKey(); - OperationStackContext stackCtx = default; - if (IsTailmostMainKvRecord(ref key, recordInfo, ref stackCtx)) + OperationStackContext stackCtx = default; + if (IsTailmostMainKvRecord(mainKvIter.Key, mainKvIter.Info, ref stackCtx)) return true; - ProcessNonTailmostMainKvRecord(recordInfo, key); + ProcessNonTailmostMainKvRecord(mainKvIter.Info, mainKvIter.Key); continue; } @@ -140,9 +135,9 @@ public bool GetNext(out RecordInfo recordInfo) if (iterationPhase == IterationPhase.TempKv) { - if (tempKvIter.GetNext(out recordInfo)) + if (tempKvIter.GetNext()) { - if (!recordInfo.Tombstone) + if (!tempKvIter.Info.Tombstone) return true; continue; } @@ -153,35 +148,31 @@ public bool GetNext(out RecordInfo recordInfo) } // We're done. This handles both the call that exhausted tempKvIter, and any subsequent calls on this outer iterator. - recordInfo = default; return false; } } internal bool PushNext(ref TScanFunctions scanFunctions, long numRecords, out bool stop) - where TScanFunctions : IScanIteratorFunctions + where TScanFunctions : IScanIteratorFunctions { while (true) { if (iterationPhase == IterationPhase.MainKv) { - OperationStackContext stackCtx = default; - if (mainKvIter.GetNext(out var recordInfo)) + OperationStackContext stackCtx = default; + if (mainKvIter.GetNext()) { try { - ref var key = ref mainKvIter.GetKey(); - if (IsTailmostMainKvRecord(ref key, recordInfo, ref stackCtx)) + var key = mainKvIter.Key; + if (IsTailmostMainKvRecord(key, mainKvIter.Info, ref stackCtx)) { - // Push Iter records are in temp storage so do not need locks, but we'll call ConcurrentReader because, for example, GenericAllocator - // may need to know the object is in that region. - stop = mainKvIter.CurrentAddress >= store.hlogBase.ReadOnlyAddress - ? !scanFunctions.ConcurrentReader(ref key, ref mainKvIter.GetValue(), new RecordMetadata(recordInfo, mainKvIter.CurrentAddress), numRecords, out _) - : !scanFunctions.SingleReader(ref key, ref mainKvIter.GetValue(), new RecordMetadata(recordInfo, mainKvIter.CurrentAddress), numRecords, out _); + // Push Iter records are in temp storage so do not need locks. + stop = !scanFunctions.Reader(ref mainKvIter, new RecordMetadata(mainKvIter.CurrentAddress), numRecords, out _); return !stop; } - ProcessNonTailmostMainKvRecord(recordInfo, key); + ProcessNonTailmostMainKvRecord(mainKvIter.Info, key); continue; } catch (Exception ex) @@ -204,11 +195,11 @@ internal bool PushNext(ref TScanFunctions scanFunctions, long nu if (iterationPhase == IterationPhase.TempKv) { - if (tempKvIter.GetNext(out var recordInfo)) + if (tempKvIter.GetNext()) { - if (!recordInfo.Tombstone) + if (!tempKvIter.Info.Tombstone) { - stop = !scanFunctions.SingleReader(ref tempKvIter.GetKey(), ref tempKvIter.GetValue(), new RecordMetadata(recordInfo, tempKvIter.CurrentAddress), numRecords, out _); + stop = !scanFunctions.Reader(ref tempKvIter, new RecordMetadata(tempKvIter.CurrentAddress), numRecords, out _); return !stop; } continue; @@ -225,23 +216,26 @@ internal bool PushNext(ref TScanFunctions scanFunctions, long nu } } - private void ProcessNonTailmostMainKvRecord(RecordInfo recordInfo, TKey key) + private void ProcessNonTailmostMainKvRecord(RecordInfo recordInfo, ReadOnlySpan key) { // Not the tailmost record in the tag chain so add it to or remove it from tempKV (we want to return only the latest version). if (recordInfo.Tombstone) { // Check if it's in-memory first so we don't spuriously create a tombstone record. - if (tempbContext.ContainsKeyInMemory(ref key, out _).Found) - _ = tempbContext.Delete(ref key); + if (tempbContext.ContainsKeyInMemory(key, out _).Found) + _ = tempbContext.Delete(key); } else - _ = tempbContext.Upsert(ref key, ref mainKvIter.GetValue()); + { + var iterLogRecord = mainKvIter as ISourceLogRecord; // Can't use 'ref' on a 'using' variable + _ = tempbContext.Upsert(ref iterLogRecord); + } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IsTailmostMainKvRecord(ref TKey key, RecordInfo mainKvRecordInfo, ref OperationStackContext stackCtx) + bool IsTailmostMainKvRecord(ReadOnlySpan key, RecordInfo mainKvRecordInfo, ref OperationStackContext stackCtx) { - stackCtx = new(store.storeFunctions.GetKeyHashCode64(ref key)); + stackCtx = new(store.storeFunctions.GetKeyHashCode64(key)); if (store.FindTag(ref stackCtx.hei)) { stackCtx.SetRecordSourceToHashEntry(store.hlogBase); @@ -253,8 +247,8 @@ bool IsTailmostMainKvRecord(ref TKey key, RecordInfo mainKvRecordInfo, ref Opera if (mainKvRecordInfo.PreviousAddress >= store.Log.BeginAddress) { // Check if it's in-memory first so we don't spuriously create a tombstone record. - if (tempbContext.ContainsKeyInMemory(ref key, out _).Found) - tempbContext.Delete(ref key); + if (tempbContext.ContainsKeyInMemory(key, out _).Found) + _ = tempbContext.Delete(key); } // If the record is not deleted, we can let the caller process it directly within mainKvIter. @@ -264,26 +258,57 @@ bool IsTailmostMainKvRecord(ref TKey key, RecordInfo mainKvRecordInfo, ref Opera return false; } - public bool GetNext(out RecordInfo recordInfo, out TKey key, out TValue value) - { - if (GetNext(out recordInfo)) - { - if (iterationPhase == IterationPhase.MainKv) - { - key = mainKvIter.GetKey(); - value = mainKvIter.GetValue(); - } - else - { - key = tempKvIter.GetKey(); - value = tempKvIter.GetValue(); - } - return true; - } + #region ISourceLogRecord + /// + public ref RecordInfo InfoRef => ref CurrentIter.InfoRef; + /// + public RecordInfo Info => CurrentIter.Info; - key = default; - value = default; - return false; - } + /// + public bool IsSet => !CurrentIter.IsSet; + + /// + public ReadOnlySpan Key => CurrentIter.Key; + + /// + public bool IsPinnedKey => CurrentIter.IsPinnedKey; + + /// + public unsafe byte* PinnedKeyPointer => CurrentIter.PinnedKeyPointer; + + /// + public unsafe Span ValueSpan => CurrentIter.ValueSpan; + + /// + public IHeapObject ValueObject => CurrentIter.ValueObject; + + /// + public ReadOnlySpan RecordSpan => CurrentIter.RecordSpan; + + /// + public bool IsPinnedValue => CurrentIter.IsPinnedValue; + + /// + public unsafe byte* PinnedValuePointer => CurrentIter.PinnedValuePointer; + + /// + public long ETag => CurrentIter.ETag; + + /// + public long Expiration => CurrentIter.Expiration; + + /// + public void ClearValueObject(Action disposer) { } // Not relevant for iterators + + /// + public bool AsLogRecord(out LogRecord logRecord) => CurrentIter.AsLogRecord(out logRecord); + + /// + public bool AsDiskLogRecord(out DiskLogRecord diskLogRecord) => CurrentIter.AsDiskLogRecord(out diskLogRecord); + + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public RecordFieldInfo GetRecordFieldInfo() => CurrentIter.GetRecordFieldInfo(); + #endregion // ISourceLogRecord } } \ No newline at end of file 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 e0d7e6a54f2..293a9724162 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -7,13 +7,13 @@ namespace Tsavorite.core { - public partial class TsavoriteKV : TsavoriteBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public partial class TsavoriteKV : TsavoriteBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void InternalRefresh(TSessionFunctionsWrapper sessionFunctions) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { epoch.ProtectAndDrain(); @@ -40,7 +40,7 @@ internal void InternalRefresh(TSessionFunctionsWrapper sessionFunctions, bool wait = false, - CompletedOutputIterator completedOutputs = null) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + CompletedOutputIterator completedOutputs = null) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { while (true) { InternalCompletePendingRequests(sessionFunctions, completedOutputs); if (wait) sessionFunctions.Ctx.WaitPending(epoch); - if (sessionFunctions.Ctx.HasNoPendingRequests) return true; + if (sessionFunctions.Ctx.HasNoPendingRequests) + return true; InternalRefresh(sessionFunctions); - if (!wait) return false; - Thread.Yield(); + if (!wait) + return false; + _ = Thread.Yield(); } } @@ -74,20 +76,20 @@ internal bool InternalCompletePending(TSessionFunctionsWrapper sessionFunctions, - CompletedOutputIterator completedOutputs) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + CompletedOutputIterator completedOutputs) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { _ = hlogBase.TryComplete(); if (sessionFunctions.Ctx.readyResponses.Count == 0) return; - while (sessionFunctions.Ctx.readyResponses.TryDequeue(out AsyncIOContext request)) + while (sessionFunctions.Ctx.readyResponses.TryDequeue(out AsyncIOContext request)) InternalCompletePendingRequest(sessionFunctions, request, completedOutputs); } - internal void InternalCompletePendingRequest(TSessionFunctionsWrapper sessionFunctions, AsyncIOContext request, - CompletedOutputIterator completedOutputs) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal void InternalCompletePendingRequest(TSessionFunctionsWrapper sessionFunctions, AsyncIOContext request, + CompletedOutputIterator completedOutputs) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { // Get and Remove this request.id pending dictionary if it is there. if (sessionFunctions.Ctx.ioPendingRequests.Remove(request.id, out var pendingContext)) @@ -96,7 +98,7 @@ internal void InternalCompletePendingRequest /// Caller is expected to dispose pendingContext after this method completes /// - internal Status InternalCompletePendingRequestFromContext(TSessionFunctionsWrapper sessionFunctions, AsyncIOContext request, - ref PendingContext pendingContext, out AsyncIOContext newRequest) - where TSessionFunctionsWrapper : ISessionFunctionsWrapper + internal unsafe Status InternalCompletePendingRequestFromContext(TSessionFunctionsWrapper sessionFunctions, AsyncIOContext request, + ref PendingContext pendingContext, out AsyncIOContext newRequest) + where TSessionFunctionsWrapper : ISessionFunctionsWrapper { Debug.Assert(epoch.ThisInstanceProtected(), "InternalCompletePendingRequestFromContext requires epoch acquision"); newRequest = default; // If NoKey, we do not have the key in the initial call and must use the key from the satisfied request. - // With the new overload of CompletePending that returns CompletedOutputs, pendingContext must have the key. - if (pendingContext.NoKey && pendingContext.key == default) - pendingContext.key = hlog.GetKeyContainer(ref hlog.GetContextRecordKey(ref request)); - ref TKey key = ref pendingContext.key.Get(); + // Otherwise the key is already in the pendingContext *and* the key in diskLogRecord has been verified to match. + DiskLogRecord diskLogRecord = new(ref request); + var key = diskLogRecord.Key; OperationStatus internalStatus = pendingContext.type switch { @@ -135,29 +136,25 @@ internal Status InternalCompletePendingRequestFromContext - /// The reason a SingleWriter was performed - /// - public enum WriteReason : byte - { - /// A new record appended by Upsert - Upsert, - - /// Copying a read from disk to the tail of the log - CopyToTail, - - /// Copying a read from disk to the read cache - CopyToReadCache, - - /// The user called Compact() - Compaction - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Tsavorite.core.csproj b/libs/storage/Tsavorite/cs/src/core/Tsavorite.core.csproj index b91f3e8132d..d180feda2ba 100644 --- a/libs/storage/Tsavorite/cs/src/core/Tsavorite.core.csproj +++ b/libs/storage/Tsavorite/cs/src/core/Tsavorite.core.csproj @@ -11,6 +11,22 @@ ;NU1605 + + 1701;1702;IDE0130 + + + + 1701;1702;IDE0130 + + + + 1701;1702;IDE0130 + + + + 1701;1702;IDE0130 + + diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/ILogCommitManager.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/ILogCommitManager.cs index 7d561d11eb7..e48ed668e04 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/ILogCommitManager.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/ILogCommitManager.cs @@ -7,7 +7,7 @@ namespace Tsavorite.core { /// - /// Log commit manager + /// TsavoriteLog commit manager /// public interface ILogCommitManager : IDisposable { @@ -41,7 +41,7 @@ public interface ILogCommitManager : IDisposable public void RemoveCommit(long commitNum); /// - /// Remove all log commits from this manager + /// Remove all TsavoriteLog commits from this manager /// public void RemoveAllCommits(); diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/LogCommitPolicy.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/LogCommitPolicy.cs index ba9c0a5774a..a82cc6df0f0 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/LogCommitPolicy.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/LogCommitPolicy.cs @@ -50,7 +50,7 @@ public abstract class LogCommitPolicy /// return false. /// /// policy object - public static LogCommitPolicy Default() => new DefaultLogCommitPolicy(); + public static LogCommitPolicy Default() => new DefaulLogCommitPolicy(); /// /// MaxParallel log commit policy allows k (non-strong) commit requests to be in progress at any giving time. The k commits are guaranteed @@ -76,7 +76,7 @@ public abstract class LogCommitPolicy public static LogCommitPolicy RateLimit(long thresholdMilli, long thresholdBytes) => new RateLimitLogCommitPolicy(thresholdMilli, thresholdBytes); } - internal sealed class DefaultLogCommitPolicy : LogCommitPolicy + internal sealed class DefaulLogCommitPolicy : LogCommitPolicy { /// public override void OnAttached(TsavoriteLog log) { } diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs index 9cacbed79a9..ff76c889666 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs @@ -13,16 +13,14 @@ namespace Tsavorite.core { - using EmptyStoreFunctions = StoreFunctions>; - /// - /// Tsavorite log + /// Tsavorite Log /// public sealed class TsavoriteLog : IDisposable { private Exception cannedException = null; - readonly BlittableAllocatorImpl allocator; + readonly TsavoriteLogAllocatorImpl allocator; readonly LightEpoch epoch; readonly ILogCommitManager logCommitManager; readonly bool disposeLogCommitManager; @@ -208,10 +206,7 @@ private TsavoriteLog(TsavoriteLogSettings logSettings, bool syncRecover, ILogger CommittedBeginAddress = Constants.kFirstValidAddress; SafeTailAddress = Constants.kFirstValidAddress; commitQueue = new WorkQueueLIFO(SerialCommitCallbackWorker); - allocator = new( - new AllocatorSettings(logSettings.GetLogSettings(), epoch, logger) { flushCallback = CommitCallback }, - StoreFunctions.Create(EmptyKeyComparer.Instance), - @this => new BlittableAllocator(@this)); + allocator = new(new AllocatorSettings(logSettings.GetLogSettings(), epoch, logger) { flushCallback = CommitCallback }); allocator.Initialize(); beginAddress = allocator.BeginAddress; @@ -482,7 +477,7 @@ public void Dispose() } /// - /// Mark the log as complete. A completed log will no longer allow enqueues, and all currently enqueued items will + /// Mark the log as complete. A completed log log will no longer allow enqueues, and all currently enqueued items will /// be immediately committed. /// /// whether to spin until log completion becomes committed @@ -511,7 +506,7 @@ public void CompleteLog(bool spinWait = false) } /// - /// Check if the log is complete. A completed log will no longer allow enqueues, and all currently enqueued items will + /// Check if the log is complete. A completed log log will no longer allow enqueues, and all currently enqueued items will /// be immediately committed. /// public bool LogCompleted => commitNum == long.MaxValue; @@ -522,7 +517,7 @@ internal void TrueDispose() safeTailRefreshCallbackCompleted?.Signal(); safeTailRefreshEntryEnqueued?.Signal(); commitQueue.Dispose(); - commitTcs.TrySetException(new ObjectDisposedException("Log has been disposed")); + commitTcs.TrySetException(new ObjectDisposedException("TsavoriteLog has been disposed")); allocator.Dispose(); epoch.Dispose(); if (disposeLogCommitManager) @@ -870,16 +865,16 @@ public unsafe void Enqueue(THeader userHeader, out long logicalAddress) } /// - /// Append a user-defined blittable struct header and one entry atomically to the log. + /// Append a user-defined blittable struct header and one entry atomically to the log. /// /// /// /// Logical address of added entry - public unsafe void Enqueue(THeader userHeader, ref SpanByte item, out long logicalAddress) + public unsafe void Enqueue(THeader userHeader, ReadOnlySpan item, out long logicalAddress) where THeader : unmanaged { logicalAddress = 0; - var length = sizeof(THeader) + item.TotalSize; + var length = sizeof(THeader) + item.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -889,7 +884,8 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item, out l var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item.CopyTo(physicalAddress + headerSize + sizeof(THeader)); + var offset = headerSize + sizeof(THeader); + item.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -897,17 +893,17 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item, out l } /// - /// Append a user-defined blittable struct header and two entries entries atomically to the log. + /// Append a user-defined blittable struct header and two entries entries atomically to the log. /// /// /// /// /// Logical address of added entry - public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref SpanByte item2, out long logicalAddress) + public unsafe void Enqueue(THeader userHeader, ReadOnlySpan item1, ReadOnlySpan item2, out long logicalAddress) where THeader : unmanaged { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + item2.TotalSize; + var length = sizeof(THeader) + item1.TotalSize() + item2.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -917,8 +913,10 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - item2.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -926,15 +924,15 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref } /// - /// Append two entries entries atomically to the log. + /// Append two entries entries atomically to the log. /// /// /// /// Logical address of added entry - public unsafe void Enqueue(ref SpanByte item1, ref SpanByte item2, out long logicalAddress) + public unsafe void Enqueue(ReadOnlySpan item1, ReadOnlySpan item2, out long logicalAddress) { logicalAddress = 0; - var length = item1.TotalSize + item2.TotalSize; + var length = item1.TotalSize() + item2.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -943,8 +941,10 @@ public unsafe void Enqueue(ref SpanByte item1, ref SpanByte item2, out long logi logicalAddress = AllocateBlock(allocatedLength); var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); - item1.CopyTo(physicalAddress + headerSize); - item2.CopyTo(physicalAddress + headerSize + item1.TotalSize); + var offset = headerSize; + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -952,18 +952,18 @@ public unsafe void Enqueue(ref SpanByte item1, ref SpanByte item2, out long logi } /// - /// Append a user-defined blittable struct header and three entries entries atomically to the log. + /// Append a user-defined blittable struct header and three entries entries atomically to the log. /// /// /// /// /// /// Logical address of added entry - public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref SpanByte item2, ref SpanByte item3, out long logicalAddress) + public unsafe void Enqueue(THeader userHeader, ReadOnlySpan item1, ReadOnlySpan item2, ReadOnlySpan item3, out long logicalAddress) where THeader : unmanaged { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + item2.TotalSize + item3.TotalSize; + var length = sizeof(THeader) + item1.TotalSize() + item2.TotalSize() + item3.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -973,9 +973,12 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - item2.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize); - item3.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize + item2.TotalSize); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item2.TotalSize(); + item3.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -983,7 +986,7 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref } /// - /// Append a user-defined blittable struct header and three entries entries atomically to the log. + /// Append a user-defined blittable struct header and three entries entries atomically to the log. /// /// /// @@ -1009,17 +1012,17 @@ public unsafe void Enqueue(THeader userHeader, ref TInput input } /// - /// Append a user-defined blittable struct header and three entries entries atomically to the log. + /// Append a user-defined blittable struct header and three entries entries atomically to the log. /// /// /// /// /// Logical address of added entry - public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref TInput input, out long logicalAddress) + public unsafe void Enqueue(THeader userHeader, ReadOnlySpan item1, ref TInput input, out long logicalAddress) where THeader : unmanaged where TInput : IStoreInput { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + input.SerializedLength; + var length = sizeof(THeader) + item1.TotalSize() + input.SerializedLength; var allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1028,8 +1031,10 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte ite logicalAddress = AllocateBlock(allocatedLength); var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - input.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize, input.SerializedLength); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + input.CopyTo(physicalAddress + offset, input.SerializedLength); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -1037,18 +1042,18 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte ite } /// - /// Append a user-defined blittable struct header and three entries entries atomically to the log. + /// Append a user-defined blittable struct header and three entries entries atomically to the log. /// /// /// /// /// /// Logical address of added entry - public unsafe void Enqueue(THeader userHeader, ref SpanByte item1, ref SpanByte item2, ref TInput input, out long logicalAddress) + public unsafe void Enqueue(THeader userHeader, ReadOnlySpan item1, ReadOnlySpan item2, ref TInput input, out long logicalAddress) where THeader : unmanaged where TInput : IStoreInput { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + item2.TotalSize + input.SerializedLength; + var length = sizeof(THeader) + item1.TotalSize() + item2.TotalSize() + input.SerializedLength; var allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1057,10 +1062,12 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte ite logicalAddress = AllocateBlock(allocatedLength); var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - item2.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize); - input.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize + item2.TotalSize, - input.SerializedLength); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item2.TotalSize(); + input.CopyTo(physicalAddress + offset, input.SerializedLength); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -1068,15 +1075,15 @@ public unsafe void Enqueue(THeader userHeader, ref SpanByte ite } /// - /// Append a user-defined header byte and a entry atomically to the log. + /// Append a user-defined header byte and a entry atomically to the log. /// /// /// /// Logical address of added entry - public unsafe void Enqueue(byte userHeader, ref SpanByte item, out long logicalAddress) + public unsafe void Enqueue(byte userHeader, ReadOnlySpan item, out long logicalAddress) { logicalAddress = 0; - var length = sizeof(byte) + item.TotalSize; + var length = sizeof(byte) + item.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1086,7 +1093,8 @@ public unsafe void Enqueue(byte userHeader, ref SpanByte item, out long logicalA var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *physicalAddress = userHeader; - item.CopyTo(physicalAddress + sizeof(byte)); + var offset = sizeof(byte); + item.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -1120,7 +1128,7 @@ private long AllocateBlock(int recordSize) } /// - /// Try to append a user-defined blittable struct header and two entries entries atomically to the log. + /// Try to append a user-defined blittable struct header and two entries entries atomically to the log. /// If it returns true, we are done. If it returns false, we need to retry. /// /// @@ -1128,11 +1136,11 @@ private long AllocateBlock(int recordSize) /// /// Logical address of added entry /// Whether the append succeeded - public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, ref SpanByte item2, out long logicalAddress) + public unsafe bool TryEnqueue(THeader userHeader, ReadOnlySpan item1, ReadOnlySpan item2, out long logicalAddress) where THeader : unmanaged { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + item2.TotalSize; + var length = sizeof(THeader) + item1.TotalSize() + item2.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1148,8 +1156,10 @@ public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, r var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - item2.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -1158,7 +1168,7 @@ public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, r } /// - /// Try to append a user-defined blittable struct header and three entries entries atomically to the log. + /// Try to append a user-defined blittable struct header and three entries entries atomically to the log. /// If it returns true, we are done. If it returns false, we need to retry. /// /// @@ -1167,11 +1177,11 @@ public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, r /// /// Logical address of added entry /// Whether the append succeeded - public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, ref SpanByte item2, ref SpanByte item3, out long logicalAddress) + public unsafe bool TryEnqueue(THeader userHeader, ReadOnlySpan item1, ReadOnlySpan item2, ReadOnlySpan item3, out long logicalAddress) where THeader : unmanaged { logicalAddress = 0; - var length = sizeof(THeader) + item1.TotalSize + item2.TotalSize + item3.TotalSize; + var length = sizeof(THeader) + item1.TotalSize() + item2.TotalSize() + item3.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1187,9 +1197,12 @@ public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, r var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *(THeader*)(physicalAddress + headerSize) = userHeader; - item1.CopyTo(physicalAddress + headerSize + sizeof(THeader)); - item2.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize); - item3.CopyTo(physicalAddress + headerSize + sizeof(THeader) + item1.TotalSize + item2.TotalSize); + var offset = headerSize + sizeof(THeader); + item1.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item1.TotalSize(); + item2.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); + offset += item2.TotalSize(); + item3.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -1198,17 +1211,17 @@ public unsafe bool TryEnqueue(THeader userHeader, ref SpanByte item1, r } /// - /// Try to append a user-defined header byte and a entry atomically to the log. If it returns true, we are + /// Try to append a user-defined header byte and a entry atomically to the log. If it returns true, we are /// done. If it returns false, we need to retry. /// /// /// /// Logical address of added entry /// Whether the append succeeded - public unsafe bool TryEnqueue(byte userHeader, ref SpanByte item, out long logicalAddress) + public unsafe bool TryEnqueue(byte userHeader, ReadOnlySpan item, out long logicalAddress) { logicalAddress = 0; - var length = sizeof(byte) + item.TotalSize; + var length = sizeof(byte) + item.TotalSize(); int allocatedLength = headerSize + Align(length); ValidateAllocatedLength(allocatedLength); @@ -1224,7 +1237,8 @@ public unsafe bool TryEnqueue(byte userHeader, ref SpanByte item, out long logic var physicalAddress = (byte*)allocator.GetPhysicalAddress(logicalAddress); *physicalAddress = userHeader; - item.CopyTo(physicalAddress + sizeof(byte)); + var offset = sizeof(byte); + item.SerializeTo(new Span(physicalAddress + offset, allocatedLength - offset)); SetHeader(length, physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); epoch.Suspend(); @@ -2081,11 +2095,11 @@ public void TruncateUntilPageStart(long untilAddress) /// Whether we scan uncommitted data /// /// - public TsavoriteLogScanIterator Scan(long beginAddress, long endAddress, bool recover = true, ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering, bool scanUncommitted = false, ILogger logger = null) + public TsavoriteLogIterator Scan(long beginAddress, long endAddress, bool recover = true, DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering, bool scanUncommitted = false, ILogger logger = null) { if (readOnlyMode) { - scanBufferingMode = ScanBufferingMode.SinglePageBuffering; + scanBufferingMode = DiskScanBufferingMode.SinglePageBuffering; if (scanUncommitted) throw new TsavoriteException("Cannot use scanUncommitted with read-only TsavoriteLog"); @@ -2094,7 +2108,7 @@ public TsavoriteLogScanIterator Scan(long beginAddress, long endAddress, bool re if (scanUncommitted && SafeTailRefreshFrequencyMs < 0) throw new TsavoriteException("Cannot use scanUncommitted without setting SafeTailRefreshFrequencyMs to a non-negative value in TsavoriteLog settings"); - var iter = new TsavoriteLogScanIterator(this, allocator, beginAddress, endAddress, getMemory, scanBufferingMode, epoch, headerSize, scanUncommitted, logger: logger); + var iter = new TsavoriteLogIterator(this, allocator, beginAddress, endAddress, getMemory, scanBufferingMode, epoch, headerSize, scanUncommitted, logger: logger); if (Interlocked.Increment(ref logRefCount) == 1) throw new TsavoriteException("Cannot scan disposed log instance"); @@ -2123,11 +2137,11 @@ public void RemoveIterator(TsavoriteLogScanSingleIterator iterator) } } - public TsavoriteLogScanSingleIterator ScanSingle(long beginAddress, long endAddress, bool recover = true, ScanBufferingMode scanBufferingMode = ScanBufferingMode.DoublePageBuffering, bool scanUncommitted = false, ILogger logger = null) + public TsavoriteLogScanSingleIterator ScanSingle(long beginAddress, long endAddress, bool recover = true, DiskScanBufferingMode scanBufferingMode = DiskScanBufferingMode.DoublePageBuffering, bool scanUncommitted = false, ILogger logger = null) { if (readOnlyMode) { - scanBufferingMode = ScanBufferingMode.SinglePageBuffering; + scanBufferingMode = DiskScanBufferingMode.SinglePageBuffering; if (scanUncommitted) throw new TsavoriteException("Cannot use scanUncommitted with read-only TsavoriteLog"); diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs index a04f1d28749..a5d40b32e7c 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogIterator.cs @@ -12,15 +12,13 @@ namespace Tsavorite.core { - using EmptyStoreFunctions = StoreFunctions>; - /// - /// Scan iterator for hybrid log + /// Scan iterator for TsavoriteLog /// - public class TsavoriteLogScanIterator : ScanIteratorBase, IDisposable + public class TsavoriteLogIterator : ScanIteratorBase, IDisposable { protected readonly TsavoriteLog tsavoriteLog; - private readonly BlittableAllocatorImpl allocator; + private readonly TsavoriteLogAllocatorImpl allocator; private readonly BlittableFrame frame; private readonly GetMemory getMemory; private readonly int headerSize; @@ -33,22 +31,10 @@ public class TsavoriteLogScanIterator : ScanIteratorBase, IDisposable /// public bool Ended => (nextAddress >= endAddress) || (tsavoriteLog.LogCompleted && nextAddress == tsavoriteLog.TailAddress); - /// - /// Constructor - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - /// - internal unsafe TsavoriteLogScanIterator(TsavoriteLog tsavoriteLog, BlittableAllocatorImpl hlog, long beginAddress, long endAddress, - GetMemory getMemory, ScanBufferingMode scanBufferingMode, LightEpoch epoch, int headerSize, bool scanUncommitted = false, ILogger logger = null) - : base(beginAddress == 0 ? hlog.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, scanBufferingMode, false, epoch, hlog.LogPageSizeBits, logger: logger) + /// 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.GetFirstValidLogicalAddress(0) : beginAddress, endAddress, diskScanBufferingMode, InMemoryScanBufferingMode.NoBuffering, includeSealedRecords: false, epoch, hlog.LogPageSizeBits, logger: logger) { this.tsavoriteLog = tsavoriteLog; allocator = hlog; @@ -76,7 +62,6 @@ internal unsafe TsavoriteLogScanIterator(TsavoriteLog tsavoriteLog, BlittableAll if (!await WaitAsync(token).ConfigureAwait(false)) yield break; } - yield return (result, length, currentAddress, nextAddress); } } @@ -163,7 +148,7 @@ public ValueTask WaitAsync(CancellationToken token = default) return SlowWaitUncommittedAsync(token); } - private static async ValueTask SlowWaitAsync(TsavoriteLogScanIterator @this, CancellationToken token) + private static async ValueTask SlowWaitAsync(TsavoriteLogIterator @this, CancellationToken token) { while (true) { @@ -428,11 +413,11 @@ public unsafe bool TryConsumeNext(T consumer) where T : ILogEntryConsumer throw; } - if (isCommitRecord) - { - TsavoriteLogRecoveryInfo info = new(); - info.Initialize(new ReadOnlySpan((byte*)(headerSize + physicalAddress), entryLength)); - if (info.CommitNum != long.MaxValue) continue; + if (isCommitRecord) + { + TsavoriteLogRecoveryInfo info = new(); + info.Initialize(new ReadOnlySpan((byte*)(headerSize + physicalAddress), entryLength)); + if (info.CommitNum != long.MaxValue) continue; // Otherwise, no more entries return false; diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogRecoveryInfo.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogRecoveryInfo.cs index 244d36f0855..ef7d5573020 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogRecoveryInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogRecoveryInfo.cs @@ -10,7 +10,7 @@ namespace Tsavorite.core { /// - /// Recovery info for Tsavorite Log + /// Recovery info for TsavoriteLog /// public struct TsavoriteLogRecoveryInfo { diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogScanSingleIterator.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogScanSingleIterator.cs index 7ffb879bff8..ea9f004c9e0 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogScanSingleIterator.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogScanSingleIterator.cs @@ -7,20 +7,18 @@ namespace Tsavorite.core { - using EmptyStoreFunctions = StoreFunctions>; - /// - /// Scan iterator for hybrid log - only a single scan is supported per instance + /// Scan iterator for TsavoriteLog's hybrid log - only a single scan is supported per instance /// This modification allows us to use a SingleWaiterAutoResetEvent per iterator /// so we can avoid TCS allocations per tail bump. /// - public sealed class TsavoriteLogScanSingleIterator : TsavoriteLogScanIterator + public sealed class TsavoriteLogScanSingleIterator : TsavoriteLogIterator { readonly SingleWaiterAutoResetEvent onEnqueue; - internal TsavoriteLogScanSingleIterator(TsavoriteLog tsavoriteLog, BlittableAllocatorImpl hlog, long beginAddress, long endAddress, - GetMemory getMemory, ScanBufferingMode scanBufferingMode, LightEpoch epoch, int headerSize, bool scanUncommitted = false, ILogger logger = null) - : base(tsavoriteLog, hlog, beginAddress, endAddress, getMemory, scanBufferingMode, epoch, headerSize, scanUncommitted, logger) + internal TsavoriteLogScanSingleIterator(TsavoriteLog TsavoriteLog, TsavoriteLogAllocatorImpl hlog, long beginAddress, long endAddress, + GetMemory getMemory, DiskScanBufferingMode scanBufferingMode, LightEpoch epoch, int headerSize, bool scanUncommitted = false, ILogger logger = null) + : base(TsavoriteLog, hlog, beginAddress, endAddress, getMemory, scanBufferingMode, epoch, headerSize, scanUncommitted, logger) { onEnqueue = new() { diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogSettings.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogSettings.cs index 63e7fa21408..be89c82c5b3 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLogSettings.cs @@ -14,7 +14,7 @@ namespace Tsavorite.core public delegate byte[] GetMemory(int minLength); /// - /// Type of checksum to add to log + /// Type of checksum to add to TsavoriteLog /// public enum LogChecksumType { @@ -29,7 +29,7 @@ public enum LogChecksumType } /// - /// Tsavorite Log Settings + /// Tsavorite Log LogSettings /// public class TsavoriteLogSettings : IDisposable { @@ -76,7 +76,7 @@ public class TsavoriteLogSettings : IDisposable public int SegmentSizeBits { set { SegmentSize = 1L << value; } } /// - /// Log commit manager - if you want to override the default implementation of commit. + /// TsavoriteLog commit manager - if you want to override the default implementation of commit. /// public ILogCommitManager LogCommitManager = null; @@ -93,7 +93,7 @@ public class TsavoriteLogSettings : IDisposable public GetMemory GetMemory = null; /// - /// Type of checksum to add to log + /// Type of checksum to add to TsavoriteLog /// public LogChecksumType LogChecksum = LogChecksumType.None; @@ -121,7 +121,7 @@ public class TsavoriteLogSettings : IDisposable public bool RemoveOutdatedCommits = true; /// - /// Log commit policy that influences the behavior of Commit() calls. + /// TsavoriteLog commit policy that influences the behavior of Commit() calls. /// public LogCommitPolicy LogCommitPolicy = LogCommitPolicy.Default(); @@ -159,7 +159,7 @@ public TsavoriteLogSettings(string baseDir, bool deleteDirOnDispose = false) disposeDevices = true; this.deleteDirOnDispose = deleteDirOnDispose; this.baseDir = baseDir; - LogDevice = baseDir == null ? new NullDevice() : Devices.CreateLogDevice(baseDir + "/tsavoritelog.log", deleteOnClose: deleteDirOnDispose); + LogDevice = baseDir == null ? new NullDevice() : Devices.CreateLogDevice(baseDir + "/TsavoriteLog.log", deleteOnClose: deleteDirOnDispose); LogCommitDir = baseDir; } diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs index 8461949ea3b..7900fec9625 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/BufferPool.cs @@ -116,6 +116,12 @@ public SectorAlignedMemory(int numRecords, int sectorSize) // Assume ctor is called for allocation and leave Free unset } + public unsafe (byte[] array, long offset) GetArrayAndUnalignedOffset(long alignedOffset) + { + long ptr = (long)Unsafe.AsPointer(ref buffer[0]); + return (buffer, alignedOffset + ptr - (long)aligned_pointer); + } + /// /// Dispose /// @@ -206,6 +212,20 @@ public SectorAlignedBufferPool(int recordSize, int sectorSize) this.sectorSize = sectorSize; } + public void EnsureSize(ref SectorAlignedMemory page, int size) + { + if (page is null) + { + page = Get(size); + return; + } + if (page.AlignedTotalCapacity < size) + { + page.Return(); + page = Get(size); + } + } + /// /// Return /// @@ -262,7 +282,7 @@ public unsafe SectorAlignedMemory Get(int numRecords) Interlocked.Increment(ref totalGets); #endif - int requiredSize = sectorSize + (((numRecords) * recordSize + (sectorSize - 1)) & ~(sectorSize - 1)); + int requiredSize = sectorSize + ((numRecords * recordSize + (sectorSize - 1)) & ~(sectorSize - 1)); int index = Position(requiredSize / sectorSize); if (queue[index] == null) { diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/LockType.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/LockType.cs index 18335efb675..7e38fdae0d1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/LockType.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/LockType.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// @@ -27,10 +29,10 @@ public enum LockType : byte /// /// Interface that must be implemented to participate in keyHash-based locking. /// - public interface ILockableKey + public interface ITransactionalKey { /// - /// The hash code for a specific key, obtained from + /// The hash code for a specific key, obtained from /// public long KeyHash { get; } @@ -43,58 +45,48 @@ public interface ILockableKey /// /// A utility class to carry a fixed-length key (blittable or object type) and its assciated info for Locking /// - /// - public struct FixedLengthLockableKeyStruct : ILockableKey + public struct FixedLengthTransactionalKeyStruct : ITransactionalKey { /// /// The key that is acquiring or releasing a lock /// - public TKey Key; + public PinnedSpanByte Key; - #region ILockableKey + #region ITransactionalKey /// public long KeyHash { get; set; } /// public LockType LockType { get; set; } - #endregion ILockableKey - - /// - /// Constructor - /// - public FixedLengthLockableKeyStruct(TKey key, LockType lockType, ITsavoriteContext context) : this(ref key, lockType, context) { } + #endregion ITransactionalKey /// /// Constructor /// - public FixedLengthLockableKeyStruct(ref TKey key, LockType lockType, ITsavoriteContext context) + public FixedLengthTransactionalKeyStruct(ReadOnlySpan key, LockType lockType, ITsavoriteContext context) { - Key = key; + Key = PinnedSpanByte.FromPinnedSpan(key); LockType = lockType; - KeyHash = context.GetKeyHash(ref key); + KeyHash = context.GetKeyHash(key); } - /// - /// Constructor - /// - public FixedLengthLockableKeyStruct(TKey key, long keyHash, LockType lockType, ILockableContext context) : this(ref key, keyHash, lockType, context) { } /// /// Constructor /// - public FixedLengthLockableKeyStruct(ref TKey key, long keyHash, LockType lockType, ILockableContext context) + public FixedLengthTransactionalKeyStruct(ReadOnlySpan key, long keyHash, LockType lockType, ITransactionalContext context) { - Key = key; + Key = PinnedSpanByte.FromPinnedSpan(key); KeyHash = keyHash; LockType = lockType; } /// - /// Sort the passed key array for use in - /// and + /// Sort the passed key array for use in + /// and /// /// /// - public static void Sort(FixedLengthLockableKeyStruct[] keys, ILockableContext context) => context.SortKeyHashes(keys); + public static void Sort(FixedLengthTransactionalKeyStruct[] keys, ITransactionalContext context) => context.SortKeyHashes(keys); /// public override string ToString() diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/MultiLevelPageArray.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/MultiLevelPageArray.cs new file mode 100644 index 00000000000..e545ec81e51 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/MultiLevelPageArray.cs @@ -0,0 +1,198 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Threading; + +namespace Tsavorite.core +{ + struct MultiLevelPageArray + { + // TODO: Make MLPA config numbers internally configurable (e.g. smaller log pages need less overhead). Should be able to do this internally + // and not expose another set of public config options. + internal const int InitialBookSizeBits = 2; + internal const int ChapterSizeBits = 10; + + internal const int InitialBookSize = 1 << InitialBookSizeBits; + internal const int ChapterSize = 1 << ChapterSizeBits; + internal const int PageIndexMask = (1 << ChapterSizeBits) - 1; + } + + /// + /// This creates a 3-d array of page vectors. This can be envisioned as a book, where the first two dimensions are infrastructure, and the third is where + /// the user-visible allocations are created. + /// + /// The first dimension is the "book", which is a collection of "chapters". + /// The second dimension is the "chapters", which is a collection of pages. + /// The third dimension is the actual pages of data which are returned to the user + /// + /// This structure is chosen so that only the "book" is grown; individual chapters are allocated as a fixed size. This means that + /// getting and clearing items in the chapter does not have to take a latch to prevent a lost update as the array is grown, as + /// would be necessary if there was only a single level of infrastructure (i.e. a growable chapter). + /// + internal class MultiLevelPageArray + { + internal TElement[][] book; + + internal int tail = -1; // Start at -1 so its Allocate() sets it to 0 + + public bool IsInitialized => book is not null; + + public int Count => tail + 1; // +1 because we start at -1 and increment before returning. + + public int Allocate() + { + for (; tail == -1; _ = Thread.Yield()) + { + // Two-step process in case the element allocation throws; book will still be null. + var newBook = new TElement[MultiLevelPageArray.InitialBookSize][]; + newBook[0] = new TElement[MultiLevelPageArray.ChapterSize]; + + // Multiple threads can hit this at the same time, so use Interlocked to ensure that only one thread sets book; + // others will just drop through to normal handling as book will have been set by that one thread. + if (Interlocked.CompareExchange(ref book, newBook, null) == null) + return tail = 0; + } + + while (true) + { + var originalTail = tail; + var originalChapter = originalTail >> MultiLevelPageArray.ChapterSizeBits; + if (originalChapter >= book.Length || book[originalChapter] is null) + { + // One or more other threads has incremented tail into a new, not-yet-allocated chapter, which means one owns the new-chapter "latch". + // Don't increment tail; just wait for that owning thread to allocate the new chapter. + _ = Thread.Yield(); + continue; + } + + // If we are here, we did not need to allocate a new chapter, or we own the new-chapter "latch". We'll return the incremented value of tail. + var newTail = Interlocked.Increment(ref tail); + var newChapter = newTail >> MultiLevelPageArray.ChapterSizeBits; + + Debug.Assert(newChapter >= originalChapter, $"newChapter {newChapter} should not be < originalChapter {originalChapter}"); + if (newChapter > originalChapter) + { + // We are on a new chapter, and possibly need to grow the book. If we incremented such that the value of newTail is the first page of the next chapter, + // then we are the first to cross the threshold to the new chapter and we "own the latch" for allocating that new chapter. If the increment put it past that, + // then someone else has the threshold increment and is allocating the new chapter, and we must wait until that chapter is allocated. + var newPage = newTail & MultiLevelPageArray.PageIndexMask; + if (newPage == 0) + { + AddChapter(newChapter); + } + else + { + // Wait for the thread that owns the new-page "latch" to allocate the new chapter. TODO: Tail is reset on OOM; need to break out of the inner loop and retry if that happens. + while (newChapter >= book.Length || book[newChapter] is null) + _ = Thread.Yield(); // TODO consider SpinWait.SpinOnce() with backoff + } + } + + Debug.Assert(newTail <= tail, $"newTail {newTail} should not be > tail {tail}"); + Debug.Assert(book[newChapter] is not null, $"Expected new chapter {newChapter} to be non-null pt 2"); + return newTail; + } + } + + /// + /// Add a chapter. has been incremented to be the next chapter after the last non-null chapter. + /// + private void AddChapter(int newChapterIndex) + { + // If the chapter is already allocated (e.g. reusing on a new Allocator page), we don't need to do anything, and especially + // must not change tail, because the caller will see a non-null chapter and will increment tail and expect that increment to persist. + if (newChapterIndex < book.Length && book[newChapterIndex] is not null) + return; + + try + { + if (newChapterIndex == book.Length) + { + // We need to grow the book. + var newBook = new TElement[book.Length * 2][]; + Array.Copy(book, newBook, book.Length); + book = newBook; + } + + book[newChapterIndex] = new TElement[MultiLevelPageArray.ChapterSize]; + } + catch + { + // Restore tail to the last index on the last non-null chapter--that is, the chapter before chapterIndex (this cannot set it to -1 because + // we allocate the book and first chapter in a different block). This lets the "increment tail into next chapter" loop execute and if there + // are threads spinning in that, or others enter after this throw, one will be released to try again (possibly also encountering OOM and throwing). + tail = (newChapterIndex << MultiLevelPageArray.ChapterSizeBits) - 1; + throw; + } + } + + public TElement this[int index] + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Get(index); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + set => Set(index, value); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TElement Get(int index) + { + Debug.Assert(index <= tail, $"index {index} out of range of tail {tail}"); + var localBook = book; // Temp copy as 'book' may be reallocated while we do this (but the chapter indexing remains unchanged and the chapter remains valid). + + var chapterIndex = index >> MultiLevelPageArray.ChapterSizeBits; + var pageIndex = index & MultiLevelPageArray.PageIndexMask; + Debug.Assert(localBook[chapterIndex] is not null, $"index {index} out of range of chapters {chapterIndex}"); + return localBook[chapterIndex][pageIndex]; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Set(int index, TElement element) + { + Debug.Assert(index <= tail, $"index {index} out of range of tail {tail}"); + var localBook = book; // Temp copy as 'book' may be reallocated while we do this (but the chapter indexing remains unchanged and the chapter remains valid). + + var chapterIndex = index >> MultiLevelPageArray.ChapterSizeBits; + var pageIndex = index & MultiLevelPageArray.PageIndexMask; + Debug.Assert(localBook[chapterIndex] is not null, $"index {index} out of range of chapters {chapterIndex}"); + localBook[chapterIndex][pageIndex] = element; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Clear() + { + if (!IsInitialized) + return; + var lastChapterIndex = tail >> MultiLevelPageArray.ChapterSizeBits; + for (int chapter = 0; chapter <= lastChapterIndex; chapter++) + Array.Clear(book[chapter], 0, MultiLevelPageArray.ChapterSize); + tail = 0; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Clear(Action action) + { + if (!IsInitialized) + return; + var lastChapterIndex = tail >> MultiLevelPageArray.ChapterSizeBits; + var lastPageIndex = tail & MultiLevelPageArray.PageIndexMask; + for (int chapter = 0; chapter <= lastChapterIndex; ++chapter) + { + var maxPage = chapter < lastChapterIndex ? MultiLevelPageArray.ChapterSize : lastPageIndex; + for (int page = 0; page < maxPage; ++page) + { + // Note: 'action' must check for null/default. + action(book[chapter][page]); + book[chapter][page] = default; + } + } + tail = 0; + } + + /// + public override string ToString() => $"Tail: {tail}, IsInitialized: {IsInitialized}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/SimpleConcurrentStack.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/SimpleConcurrentStack.cs new file mode 100644 index 00000000000..1b3b6a1456a --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/SimpleConcurrentStack.cs @@ -0,0 +1,193 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using System.Threading; + +namespace Tsavorite.core +{ + /// + /// This is a node in the freelists, implemented as a union of two ints and a long. The long is used for Interlocks. + /// + [StructLayout(LayoutKind.Explicit)] + internal struct SimpleFreeStackNode(int slot, int version) + { + internal const int Nil = -1; + + /// The next free node in the stack, or Empty if this is the last node. + [FieldOffset(0)] + internal int Slot = slot; + + /// The slot in the main elementArray. + [FieldOffset(4)] + internal int Version = version; + + /// The word is used for Interlocked operations, containing and . + [FieldOffset(0)] + internal long word; + + internal bool IsNil => Slot == Nil; + + public override string ToString() => $"Slot {Slot}, Version {Version}, IsNil {IsNil}"; + } + + /// + /// This is a queue containing items that may be ref or value types, but does not call Dispose; if TItem is IDisposable + /// it must be owned/disposed elsewhere. + /// + /// + /// This queue does not use latches or pointers. Instead it uses int indexes into the elementArray, and a version number + /// to avoid the ABA issue. This does mean that each item in the array is a struct containing the item and the node information, + /// so is 8 bytes (2 ints) larger than the item alone; we need to track 'next' indexes explicitly rather than rely on push/pop + /// ordering because CAS contention will alter that order. This space overhead is a tradeoff for avoiding the ABA issue without the + /// overhead of ConcurrentStack allocations or latches. + /// + class SimpleConcurrentStack + { + internal struct ArrayElement + { + internal TItem Item; + internal SimpleFreeStackNode Node; + + public override readonly string ToString() => $"[Node {Node}]; Item {Item}"; + } + + public const int DefaultInitialCapacity = 1024; + + /// The actual stack, as a simple growable vector + internal MultiLevelPageArray elementArray; + + /// + /// This is the head of the chain of stack nodes, which are used to track the stack slots in the elementArray. + /// + internal SimpleFreeStackNode stack; + + /// + /// This is the head of the chain of free nodes, which are used to track the free slots in the elementArray. + /// + internal SimpleFreeStackNode freeNodes; + + public SimpleConcurrentStack() + { + elementArray = new(); + stack = new(SimpleFreeStackNode.Nil, version: 0); + freeNodes = new(SimpleFreeStackNode.Nil, version: 0); + } + + /// + /// This is not "Count" because our and do not adjust 's Tail. + /// So we support just the high-water mark (mostly for test). + /// + public int MaxCount => elementArray.tail + 1; // +1 because we start at -1 and increment before pushing + + public bool IsEmpty => stack.IsNil; + + /// + /// Public API: Push an item onto the stack. + /// + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Push(TItem item) + { + if (GetNodeFromFreeList(out SimpleFreeStackNode node)) + ++node.Version; + else + node = new(elementArray.Allocate(), version: 0); + + // This node is going onto the stack so create an ArrayElement with its item set to the passed item and the node's version. + // We'll update the element's slot to the stack head inside the retry loop. + var element = new ArrayElement { Item = item, Node = new(SimpleFreeStackNode.Nil, node.Version) }; + + for (; ; _ = Thread.Yield()) + { + // The element's slot is the 'next' pointer; update it to what is currently in 'head' to maintain the chain. + var head = stack; + element.Node.Slot = head.Slot; + elementArray.Set(node.Slot, element); + + if (Interlocked.CompareExchange(ref stack.word, node.word, head.word) == head.word) + return; + } + } + + /// + /// Public API: Pop an item from the stack. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool TryPop(out TItem item) + { + for (; ; _ = Thread.Yield()) + { + var current = stack; + if (current.IsNil) + { + item = default; + return false; + } + + // For an element in elementArray at slot elementSlot, its node.Slot refers to the next slot in the chain; node.Version is for the current slot, + // which is also the version for the node whose .Slot == elementSlot. + var element = elementArray[current.Slot]; + var nextSlot = element.Node.Slot; + var nextVersion = element.Node.IsNil ? 0 : elementArray[nextSlot].Node.Version; + + var next = new SimpleFreeStackNode(nextSlot, nextVersion); + if (Interlocked.CompareExchange(ref stack.word, next.word, current.word) == current.word) + { + item = element.Item; + AddNodeToFreeList(current); + return true; + } + } + } + + /// Put a node that was popped from the onto the . + void AddNodeToFreeList(SimpleFreeStackNode node) + { + ++node.Version; + + // This node is going onto the freeList so create an ArrayElement with its item set to 'default' and the node's version. + // We'll update the element's slot to the freeList head inside the retry loop. + var element = new ArrayElement { Node = new(SimpleFreeStackNode.Nil, node.Version) }; + + for (; ; _ = Thread.Yield() ) + { + // The element's slot is the 'next' pointer; update it to what is currently in 'head' to maintain the chain. + var head = freeNodes; + element.Node.Slot = head.Slot; + elementArray.Set(node.Slot, element); + + if (Interlocked.CompareExchange(ref freeNodes.word, node.word, head.word) == head.word) + return; + } + } + + bool GetNodeFromFreeList(out SimpleFreeStackNode node) + { + for (; ; _ = Thread.Yield()) + { + node = freeNodes; + if (node.IsNil) + { + node = default; + return false; + } + + // For elementArray[elementSlot], node.Slot refers to the next slot in the chain; node.Version is for the current slot, which is also + // the version for the node whose .Slot == elementSlot. + var element = elementArray.Get(node.Slot); + var version = element.Node.IsNil ? 0 : elementArray.Get(element.Node.Slot).Node.Version; + + var head = new SimpleFreeStackNode(element.Node.Slot, version); + if (Interlocked.CompareExchange(ref freeNodes.word, head.word, node.word) == node.word) + return true; + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Clear() => elementArray.Clear(); + + public override string ToString() => $"elements {elementArray.Count}; [stack {stack}]; [freeList {freeNodes}]"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs index 45d584f4d0e..996181e85c2 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/StatusCode.cs @@ -18,18 +18,18 @@ internal enum StatusCode : byte /// /// /// - /// Upsert ConcurrentWriter: | + /// Upsert InPlaceWriter: | /// RMW InPlaceUpdater: | /// RMW CopyUpdater: | /// /// If NeedCopyUpdate returns false: /// - /// Delete ConcurrentDeleter: | - /// Read ConcurrentReader: + /// Delete InPlaceDeleter: | + /// Read Reader: /// /// If in immutable region and copying to tail: | /// - /// Read Pending to SingleReader: + /// Read Pending to Reader: /// /// If copying to tail: | /// If copying to readCache: | @@ -44,12 +44,12 @@ internal enum StatusCode : byte /// /// /// - /// Upsert SingleWriter (not found in mutable region): | + /// Upsert InitialWriter (not found in mutable region): | /// RMW InitialUpdater (not found in mutable, immutable, or on-disk regions): | /// /// If NeedInitialUpdate returns false: /// - /// Delete SingleDeleter (not found in mutable region): | + /// Delete InitialDeleter (not found in mutable region): | /// /// NotFound = 0x01, diff --git a/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs b/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs index a6393554874..ae1aae618b9 100644 --- a/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs +++ b/libs/storage/Tsavorite/cs/src/core/Utilities/Utility.cs @@ -75,8 +75,6 @@ internal static int NumBitsPreviousPowerOf2(long v, ILogger logger = null) /// /// Previous power of 2 /// - /// - /// internal static long PreviousPowerOf2(long v) { v |= v >> 1; @@ -88,6 +86,21 @@ internal static long PreviousPowerOf2(long v) return v - (v >> 1); } + /// + /// Next power of 2 + /// + internal static long NextPowerOf2(long v) + { + v--; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + v |= v >> 32; + return v + 1; + } + /// /// Pretty print value /// @@ -170,27 +183,25 @@ public static long GetHashCode(long input) } /// - /// Get 64-bit hash code for a byte array + /// Get 64-bit hash code for a byte array. The array does not have to be pinned. /// - /// - /// - /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static unsafe long HashBytes(byte* pbString, int len) + public static long HashBytes(ReadOnlySpan byteSpan) { const long magicno = 40343; - char* pwString = (char*)pbString; - int cbBuf = len / 2; - ulong hashState = (ulong)len; - for (int i = 0; i < cbBuf; i++, pwString++) - hashState = magicno * hashState + *pwString; + // Convert to char for faster enumeration (two bytes per iteration) + var charSpan = byteSpan.UncheckedCast(); + var hashState = (ulong)byteSpan.Length; - if ((len & 1) > 0) - { - byte* pC = (byte*)pwString; - hashState = magicno * hashState + *pC; - } + // Explicit enumerator calls are faster than foreach + var charEnumerator = charSpan.GetEnumerator(); + while (charEnumerator.MoveNext()) + hashState = (magicno * hashState) + charEnumerator.Current; + + // If we had an odd number of bytes, get the last byte + if ((byteSpan.Length & 1) > 0) + hashState = magicno * hashState + byteSpan[^1]; return (long)Rotr64(magicno * hashState, 4); } @@ -361,5 +372,11 @@ internal static string GetHashString(long hash) } internal static string GetHashString(long? hash) => hash.HasValue ? GetHashString(hash.Value) : "null"; + + /// + /// Should only be called in Debug.Assert or other DEBUG-conditional code + /// + [MethodImpl(MethodImplOptions.NoInlining)] + internal static string GetCurrentMethodName() => new StackTrace().GetFrame(1).GetMethod().Name; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/IVariableLengthInput.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/IVariableLengthInput.cs index 121a051d53c..4b60d73783d 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/IVariableLengthInput.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/IVariableLengthInput.cs @@ -1,31 +1,30 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// - /// Interface for variable length Inputs to RMW; only implemented for of . + /// Interface for variable length Inputs to Upsert and RMW. /// - public interface IVariableLengthInput + public interface IVariableLengthInput { - /// - /// Length of resulting value object when performing RMW modification of value using given input - /// - int GetRMWModifiedValueLength(ref TValue value, ref TInput input); + /// Length of resulting value object when performing RMW modification of value using given input + RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord; + + /// Initial expected length of value object when populated by RMW using given input + RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TInput input); + + /// Length of value object, when populated by Upsert using given value and input + RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref TInput input); - /// - /// Initial expected length of value object when populated by RMW using given input - /// - /// - /// - int GetRMWInitialValueLength(ref TInput input); + /// Length of value object, when populated by Upsert using given value and input + RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TInput input); - /// - /// Length of value object, when populated by Upsert using given value and input - /// - /// - /// - /// - int GetUpsertValueLength(ref TValue value, ref TInput input); + /// Length of value object, when populated by Upsert using given log record + RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ref TSourceLogRecord inputLogRecord, ref TInput input) + where TSourceLogRecord : ISourceLogRecord; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByte.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByte.cs new file mode 100644 index 00000000000..1556a4af7ed --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByte.cs @@ -0,0 +1,184 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Buffers; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using System.Text; + +namespace Tsavorite.core +{ + /// + /// Represents contiguous region of arbitrary _pinned_ memory. + /// + /// + /// SAFETY: This type is used to represent arguments that are assumed to point to pinned memory. + /// + [StructLayout(LayoutKind.Explicit, Size = Size)] + public unsafe struct PinnedSpanByte + { + public const int Size = 12; + + [FieldOffset(0)] + public byte* ptr; + + [FieldOffset(8)] + public int length; + + /// + /// Get and set length of ArgSlice. TODO: Replace length and pointer field accesses with properties + /// + public int Length + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + readonly get => length; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + set => length = value; + } + + /// Correlates to ReadOnlySpan.IsEmpty + public readonly bool IsEmpty => Length == 0; + + /// + /// Get pointer to the start of the slice + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly byte* ToPointer() => ptr; + + /// + /// Get pointer to the start of the slice + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void SetPointer(byte *newPtr) => ptr = newPtr; + + /// + /// Reset the contained Span + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Set(byte* newPtr, int newLength) + { + ptr = newPtr; + length = newLength; + } + + /// + /// Total size of the contained span, including the length prefix. + /// + public readonly int TotalSize + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return sizeof(int) + length; } + } + + /// + /// Set this as invalid; used by to indicate the should be used. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Invalidate() => ptr = null; + + /// + /// If the pointer is null, this PinnedSpanByte is not valid + /// + public readonly bool IsValid + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return ptr != null; } + } + + /// + /// Defines an implicit conversion to a + /// + public static implicit operator ReadOnlySpan(PinnedSpanByte psb) => psb.ReadOnlySpan; + + /// + /// Get slice as ReadOnlySpan + /// + public readonly ReadOnlySpan ReadOnlySpan + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get => new(ptr, length); + } + + /// + /// Get slice as Span + /// + public readonly Span Span + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new(ptr, length); } + } + + /// + /// Copies the contents of this slice into a new array. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly byte[] ToArray() => ReadOnlySpan.ToArray(); + + /// + /// Decodes the contents of this slice as ASCII into a new string. + /// + /// A string ASCII decoded string from the slice. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public override readonly string ToString() => IsValid ? Encoding.ASCII.GetString(ReadOnlySpan) : $", len {Length}"; + + /// + /// Create a from the given . + /// + /// + /// SAFETY: The MUST point to pinned memory. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static PinnedSpanByte FromPinnedSpan(ReadOnlySpan span) + => FromPinnedPointer((byte*)Unsafe.AsPointer(ref MemoryMarshal.GetReference(span)), span.Length); + + /// + /// Create new ArgSlice from given pointer and length + /// + /// + /// SAFETY: The MUST point to pinned memory. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static PinnedSpanByte FromPinnedPointer(byte* ptr, int length) + => new() { ptr = ptr, length = length }; + + /// + /// Create a SpanByte around a pinned memory whose first sizeof(int) bytes are the length (i.e. serialized form). + /// + /// + /// SAFETY: The MUST point to pinned memory. + /// + public static PinnedSpanByte FromLengthPrefixedPinnedPointer(byte* pointer) => new() { ptr = pointer + sizeof(int), length = *(int*)pointer }; + + /// + /// Check for equality to the provided argSlice + /// + /// + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public readonly bool Equals(PinnedSpanByte argSlice) => argSlice.Span.SequenceEqual(Span); + + /// + /// Copy serialized version to specified memory location + /// + /// + /// SAFETY: The MUST point to pinned memory of at least length. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void SerializeTo(byte* destination) + { + *(int*)destination = length; + Buffer.MemoryCopy(ptr, destination + sizeof(int), Length, Length); + } + + /// + /// Copy non-serialized version to specified memory location (do not copy the length prefix space) + /// + public void CopyTo(Span destination) => ReadOnlySpan.CopyTo(destination); + + /// + /// Copy non-serialized version to specified (do not copy the length prefix space) + /// + public void CopyTo(ref SpanByteAndMemory dst, MemoryPool memoryPool) => ReadOnlySpan.CopyTo(ref dst, memoryPool); + } +} \ No newline at end of file diff --git a/libs/server/ArgSlice/ArgSliceComparer.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByteComparer.cs similarity index 66% rename from libs/server/ArgSlice/ArgSliceComparer.cs rename to libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByteComparer.cs index 677c2cb6c7d..9bf68240c28 100644 --- a/libs/server/ArgSlice/ArgSliceComparer.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/PinnedSpanByteComparer.cs @@ -1,33 +1,33 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Collections.Generic; using System.Diagnostics.CodeAnalysis; using System.Numerics; -namespace Garnet.server +namespace Tsavorite.core { /// - /// ArgSlice Comparer + /// ArgSlice Comparer. This implements , not , + /// as it is not used for Tsavorite keys directly (they are converted to ). /// - public sealed class ArgSliceComparer : IEqualityComparer + public sealed class PinnedSpanByteComparer : IEqualityComparer { /// /// The default instance. /// /// Used to avoid allocating new comparers. - public static readonly ArgSliceComparer Instance = new(); + public static readonly PinnedSpanByteComparer Instance = new(); /// - public bool Equals(ArgSlice x, ArgSlice y) => x.Equals(y); + public bool Equals(PinnedSpanByte x, PinnedSpanByte y) => x.Equals(y); /// - public unsafe int GetHashCode([DisallowNull] ArgSlice obj) + public unsafe int GetHashCode([DisallowNull] PinnedSpanByte obj) { fixed (byte* ptr = obj.Span) - { return (int)HashBytes(ptr, obj.Length); - } } static unsafe long HashBytes(byte* pbString, int len) diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs new file mode 100644 index 00000000000..7b836581213 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordFieldInfo.cs @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// Struct for information about fields (Value and optional fields) of a record, to determine required allocation size. + /// + public struct RecordFieldInfo + { + /// + /// The data length of the key for the new record (not including the length prefix) and may become overflow; see + /// + public int KeyDataSize; + + /// + /// The data length of the value for the new 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 + /// + /// + public int ValueDataSize; + + /// Whether the value was specified to be an object. + public bool ValueIsObject; + + /// Whether the new record will have an ETag. + public bool HasETag; + + /// Whether the new record will have an Expiration. + public bool HasExpiration; + + /// + public override string ToString() + => $"KeySize {KeyDataSize}, ValSize {ValueDataSize}, ValIsObj {ValueIsObject}, HasETag {HasETag}, HasExpir {HasExpiration}"; + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs new file mode 100644 index 00000000000..8cbeac0db83 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/RecordSizeInfo.cs @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; + +namespace Tsavorite.core +{ + /// + /// Struct for information about the key and the fields and their sizes in a record. + /// + 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. + public bool KeyIsInline; + + /// Whether the value was within the inline max value length. + public bool ValueIsInline; + + /// Whether the value was specified to be an object. + public readonly bool ValueIsObject => FieldInfo.ValueIsObject; + + /// Whether the key is an overflow allocation. + public readonly bool KeyIsOverflow => !KeyIsInline; + + /// Whether the value is an overflow allocation. + 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; + + /// 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; + + /// The max inline value size if this is a record in the string log. + public int MaxInlineValueSpanSize { 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 . + public int ActualInlineRecordSize { readonly get; internal set; } + + /// The inline size of the record rounded up to alignment. + public int AllocatedInlineRecordSize { readonly get; internal set; } + + /// Size to allocate for ETag if it will be included, else 0. + public readonly int ETagSize => FieldInfo.HasETag ? LogRecord.ETagSize : 0; + + /// Size to allocate for Expiration if it will be included, else 0. + public readonly int ExpirationSize => FieldInfo.HasExpiration ? LogRecord.ExpirationSize : 0; + + /// Size to allocate for all optional fields that will be included; possibly 0. + public readonly int OptionalSize => ETagSize + ExpirationSize; + + /// Whether these values are set (default instances are used for Delete internally, for example). + public readonly bool IsSet => AllocatedInlineRecordSize != 0; + + /// + /// 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). + /// + [Conditional("DEBUG")] + public static void AssertValueDataLength(int dataSize, ref RecordSizeInfo sizeInfo) + { + Debug.Assert(sizeInfo.FieldInfo.ValueDataSize == dataSize, $"Mismatch between expected value size {sizeInfo.FieldInfo.ValueDataSize} 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./// + [Conditional("DEBUG")] + public void AssertOptionals(RecordInfo recordInfo, bool checkETag = true, bool checkExpiration = true) + { + if (checkETag) + Debug.Assert(FieldInfo.HasETag == recordInfo.HasETag, $"Mismatch between expected HasETag {FieldInfo.HasETag} and actual ETag {recordInfo.HasETag}"); + if (checkExpiration) + Debug.Assert(FieldInfo.HasExpiration == recordInfo.HasExpiration, $"Mismatch between expected HasExpiration {FieldInfo.HasExpiration} and actual HasExpiration {recordInfo.HasExpiration}"); + } + + /// + public override readonly string ToString() + { + static string bstr(bool value) => value ? "T" : "F"; + return $"[{FieldInfo}] | KeyIsInl {bstr(KeyIsInline)}, ValIsInl {bstr(ValueIsInline)}, ValIsObj {bstr(ValueIsObject)}, ActRecSize {ActualInlineRecordSize}, AllocRecSize {AllocatedInlineRecordSize}, OptSize {OptionalSize}"; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByte.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByte.cs index 62b8d620ee0..8a1bafb1dc6 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByte.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByte.cs @@ -3,307 +3,24 @@ using System; using System.Buffers; -using System.Diagnostics; using System.Runtime.CompilerServices; using System.Runtime.InteropServices; -using System.Text; namespace Tsavorite.core { /// - /// Represents a pinned variable length byte array that is viewable as a pinned Span<byte> - /// Important: AOF header version needs to be updated if this struct's disk representation changes + /// Span{byte} static utility functions for Span{byte} and ReadOnlySpan{byte}. /// - /// - /// Format: [4-byte (int) length of payload][[optional 8-byte metadata] payload bytes...] - /// First 2 bits of length are used as a mask for properties, so max payload length is 1GB - /// - [StructLayout(LayoutKind.Explicit, Pack = 4)] - public unsafe struct SpanByte + public static unsafe class SpanByte { - // Byte #31 is used to denote unserialized (1) or serialized (0) data - private const int UnserializedBitMask = 1 << 31; - // Byte #30 is used to denote extra metadata present (1) or absent (0) in payload - private const int ExtraMetadataBitMask = 1 << 30; - // Mask for header - private const int HeaderMask = 0x3 << 30; - - /// - /// Length of the payload - /// - [FieldOffset(0)] - private int length; - - /// - /// Start of payload - /// - [FieldOffset(4)] - private IntPtr payload; - - internal readonly IntPtr Pointer => payload; - - /// - /// Pointer to the beginning of payload, not including metadata if any - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public byte* ToPointer() - { - if (Serialized) - return MetadataSize + (byte*)Unsafe.AsPointer(ref payload); - else - return MetadataSize + (byte*)payload; - } - - /// - /// Pointer to the beginning of payload, including metadata if any - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public byte* ToPointerWithMetadata() - { - if (Serialized) - return (byte*)Unsafe.AsPointer(ref payload); - else - return (byte*)payload; - } - - /// - /// Length of payload, including metadata if any - /// - public int Length - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - readonly get => length & ~HeaderMask; - [MethodImpl(MethodImplOptions.AggressiveInlining)] - set { length = (length & HeaderMask) | value; } - } - - /// - /// Length of payload, not including metadata if any - /// - public readonly int LengthWithoutMetadata => (length & ~HeaderMask) - MetadataSize; - - /// - /// Format of structure - /// - public readonly bool Serialized => (length & UnserializedBitMask) == 0; - - /// - /// Total serialized size in bytes, including header and metadata if any - /// - public readonly int TotalSize => sizeof(int) + Length; - - /// - /// Size of metadata header, if any (returns 0 or 8) - /// - public readonly int MetadataSize => (length & ExtraMetadataBitMask) >> (30 - 3); - - /// - /// Create a around a given pointer and given - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public SpanByte(int length, IntPtr payload) - { - Debug.Assert(length <= ~HeaderMask); - this.length = length | UnserializedBitMask; - this.payload = payload; - } - - /// - /// Extra metadata header - /// - public long ExtraMetadata - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - get - { - if (Serialized) - return MetadataSize > 0 ? *(long*)Unsafe.AsPointer(ref payload) : 0; - else - return MetadataSize > 0 ? *(long*)payload : 0; - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - set - { - if (value > 0) - { - length |= ExtraMetadataBitMask; - Debug.Assert(Length >= MetadataSize); - if (Serialized) - *(long*)Unsafe.AsPointer(ref payload) = value; - else - *(long*)payload = value; - } - } - } - - /// - /// Mark as having 8-byte metadata in header of payload - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void MarkExtraMetadata() - { - Debug.Assert(Length >= 8); - length |= ExtraMetadataBitMask; - } - - /// - /// Unmark as having 8-byte metadata in header of payload - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void UnmarkExtraMetadata() => length &= ~ExtraMetadataBitMask; - - /// - /// Check or set struct as invalid - /// - public bool Invalid - { - [MethodImpl(MethodImplOptions.AggressiveInlining)] - readonly get => ((length & UnserializedBitMask) != 0) && payload == IntPtr.Zero; - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - set - { - Debug.Assert(value, "Cannot restore an Invalid SpanByte to Valid; must reassign the SpanByte as a full value"); - - // Set the actual length to 0; any metadata is no longer available, and a zero length will cause callers' length checks to go - // through the ConvertToHeap path automatically. Keep the UnserializedBitMask. - length = UnserializedBitMask; - payload = IntPtr.Zero; - } - } - - /// - /// Get Span<byte> for this 's payload (excluding metadata if any) - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Span AsSpan() - { - if (Serialized) - return new Span(MetadataSize + (byte*)Unsafe.AsPointer(ref payload), Length - MetadataSize); - else - return new Span(MetadataSize + (byte*)payload, Length - MetadataSize); - } - - /// - /// Get Span<byte> for this 's payload (excluding metadata if any) - /// - /// Parameter to avoid having to call slice when wanting to interact directly with payload skipping ETag at the front of the payload - /// - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Span AsSpan(int offset) - { - if (Serialized) - return new Span(MetadataSize + (byte*)Unsafe.AsPointer(ref payload) + offset, Length - MetadataSize - offset); - else - return new Span(MetadataSize + (byte*)payload + offset, Length - MetadataSize - offset); - } - - /// - /// Get ReadOnlySpan<byte> for this 's payload (excluding metadata if any) - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ReadOnlySpan AsReadOnlySpan() - { - if (Serialized) - return new ReadOnlySpan(MetadataSize + (byte*)Unsafe.AsPointer(ref payload), Length - MetadataSize); - else - return new ReadOnlySpan(MetadataSize + (byte*)payload, Length - MetadataSize); - } - /// - /// Get ReadOnlySpan<byte> for this 's payload (excluding metadata if any) - /// - /// Parameter to avoid having to call slice when wanting to interact directly with payload skipping ETag at the front of the payload - /// - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ReadOnlySpan AsReadOnlySpan(int offset) - { - if (Serialized) - return new ReadOnlySpan(MetadataSize + (byte*)Unsafe.AsPointer(ref payload) + offset, Length - MetadataSize - offset); - else - return new ReadOnlySpan(MetadataSize + (byte*)payload + offset, Length - MetadataSize - offset); - } - - /// - /// Get Span<byte> for this 's payload (including metadata if any) - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public Span AsSpanWithMetadata() - { - if (Serialized) - return new Span((byte*)Unsafe.AsPointer(ref payload), Length); - else - return new Span((byte*)payload, Length); - } - - /// - /// Get ReadOnlySpan<byte> for this 's payload (including metadata if any) - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ReadOnlySpan AsReadOnlySpanWithMetadata() - { - if (Serialized) - return new ReadOnlySpan((byte*)Unsafe.AsPointer(ref payload), Length); - else - return new ReadOnlySpan((byte*)payload, Length); - } - - /// - /// If is in a serialized form, return a non-serialized wrapper that points to the same payload. + /// Create a Span{byte} around a stack variable. /// /// - /// SAFETY: The resulting is safe to heap-copy, as long as the underlying payload remains pinned. + /// SAFETY: The MUST be non-movable, such as on the stack. /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public SpanByte Deserialize() - { - if (!Serialized) return this; - return new SpanByte(Length - MetadataSize, (IntPtr)(MetadataSize + (byte*)Unsafe.AsPointer(ref payload))); - } - - /// - /// Reinterpret a fixed Span<byte> as a serialized . Automatically adds Span length to the first 4 bytes. - /// - public static ref SpanByte Reinterpret(Span span) - { - Debug.Assert(span.Length - sizeof(int) <= ~HeaderMask); - - fixed (byte* ptr = span) - { - *(int*)ptr = span.Length - sizeof(int); - return ref Unsafe.AsRef(ptr); - } - } - - /// - /// Reinterpret a fixed ReadOnlySpan<byte> as a serialized , without adding length header - /// - public static ref SpanByte ReinterpretWithoutLength(ReadOnlySpan span) - { - fixed (byte* ptr = span) - { - return ref Unsafe.AsRef(ptr); - } - } - - /// - /// Reinterpret a fixed pointer as a serialized - /// - public static ref SpanByte Reinterpret(byte* ptr) - { - return ref Unsafe.AsRef(ptr); - } - - /// - /// Reinterpret a fixed ref as a serialized (user needs to write the payload length to the first 4 bytes) - /// - public static ref SpanByte Reinterpret(ref T t) - { - return ref Unsafe.As(ref t); - } + public static unsafe Span FromPinnedVariable(ref T stackVar) where T : unmanaged + => new(Unsafe.AsPointer(ref stackVar), Unsafe.SizeOf()); /// /// Create a SpanByte around a pinned memory of given . @@ -311,243 +28,115 @@ public static ref SpanByte Reinterpret(ref T t) /// /// SAFETY: The MUST point to pinned memory. /// - public static SpanByte FromPinnedPointer(byte* pointer, int length) => new(length, (nint)pointer); - - /// - /// Create a SpanByte around a pinned unmanaged struct. - /// - /// - /// SAFETY: The provided unmanaged struct MUST be on the stack or point to pinned memory. - /// - public static SpanByte FromPinnedStruct(T* ptr) where T : unmanaged - => new(Unsafe.SizeOf(), (nint)ptr); - - /// - /// Create a from the given . - /// - /// - /// SAFETY: The MUST point to pinned memory. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static SpanByte FromPinnedSpan(ReadOnlySpan span) - { - return new SpanByte(span.Length, (nint)Unsafe.AsPointer(ref MemoryMarshal.GetReference(span))); - } + public static Span FromPinnedPointer(byte* pointer, int length) => new(pointer, length); /// - /// Create SpanByte around a pinned . + /// Create a SpanByte around a pinned memory whose first sizeof(int) bytes are the length (i.e. serialized form). /// /// - /// SAFETY: The MUST be pinned. + /// SAFETY: The MUST point to pinned memory. /// - public static SpanByte FromPinnedMemory(Memory memory) => FromPinnedSpan(memory.Span); - - /// - /// Convert payload to new byte array - /// - public byte[] ToByteArray() => AsReadOnlySpan().ToArray(); - - /// - /// Convert payload to specified (disposable) memory owner - /// - public (IMemoryOwner memory, int length) ToMemoryOwner(MemoryPool pool) - { - var dst = pool.Rent(Length); - AsReadOnlySpan().CopyTo(dst.Memory.Span); - return (dst, Length); - } + public static Span FromLengthPrefixedPinnedPointer(byte* pointer) => new(pointer + sizeof(int), *(int*)pointer); - /// - /// Convert to wrapper - /// - public readonly SpanByteAndMemory ToSpanByteAndMemory() => new(this); - - /// - /// Try to copy to given pre-allocated , checking if space permits at destination - /// + /// Total size, including length prefix, of a Span + /// This must be a methods instead of a property due to extension limitations [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TryCopyTo(ref SpanByte dst) - { - if (dst.Length < Length) return false; - CopyTo(ref dst); - return true; - } + public static int TotalSize(this ReadOnlySpan span) => sizeof(int) + span.Length; - /// - /// Blindly copy to given pre-allocated , assuming sufficient space. - /// Does not change length of destination. - /// + /// Total size, including length prefix, of a Span + /// This must be a methods instead of a property due to extension limitations [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CopyTo(ref SpanByte dst, long metadata = 0) - { - dst.UnmarkExtraMetadata(); - dst.ExtraMetadata = metadata == 0 ? ExtraMetadata : metadata; - AsReadOnlySpan().CopyTo(dst.AsSpan()); - } + public static int TotalSize(this Span span) => sizeof(int) + span.Length; - /// - /// Try to copy to given pre-allocated , checking if space permits at destination - /// - /// The target of the copy - /// Optional metadata to add to the destination - /// The size available at the destination (e.g. dst.TotalSize or the log-space Value allocation size) + /// Copy to given , using the Span{byte} if possible, else allocating from [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool TrySafeCopyTo(ref SpanByte dst, int fullDestSize, long metadata = 0) + public static void CopyTo(this ReadOnlySpan src, ref SpanByteAndMemory dst, MemoryPool memoryPool) { - // Need to account for extra metadata if current value does not have any. - var addMetadata = metadata > 0 && MetadataSize == 0; - - var newTotalSize = addMetadata ? TotalSize + sizeof(long) : TotalSize; - if (fullDestSize < newTotalSize) - return false; - - var newLength = addMetadata ? Length + sizeof(long) : Length; - if (dst.Length < newLength) - { - // dst is shorter than src, but we have already verified there is enough extra value space to grow dst to store src. - dst.Length = newLength; - CopyTo(ref dst, metadata); - } - else + if (dst.IsSpanByte) { - // dst length is equal or longer than src. We can adjust the length header on the serialized log, if we wish (here, we do). - // This method will also zero out the extra space to retain log scan correctness. - dst.ShrinkSerializedLength(newLength); - CopyTo(ref dst, metadata); - dst.Length = newLength; + if (dst.Length >= src.Length) + { + dst.Length = src.Length; + src.CopyTo(dst.SpanByte.Span); + return; + } + dst.ConvertToHeap(); } - return true; - } - /// - /// Shrink the length header of the in-place allocated buffer on - /// Tsavorite hybrid log, pointed to by the given . - /// Zeroes out the extra space to retain log scan correctness. - /// - /// New length of payload (including metadata) - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void ShrinkSerializedLength(int newLength) - { - // Zero-fill extra space - needed so log scan does not see spurious data - *before* setting length to 0. - if (newLength < Length) - { - Unsafe.InitBlockUnaligned(ToPointerWithMetadata() + newLength, 0, (uint)(Length - newLength)); - Length = newLength; - } + dst.Memory = memoryPool.Rent(src.Length); + dst.Length = src.Length; + src.CopyTo(dst.MemorySpan); } - /// - /// Utility to zero out an arbitrary span of bytes. - /// One use is to zero extra space after in-place update shrinks a value, to retain log scan correctness. - /// + /// Copy to given , using the Span{byte} if possible, else allocating from [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static void Clear(byte* pointer, int length) => new Span(pointer, length).Clear(); + public static void CopyTo(this Span src, ref SpanByteAndMemory dst, MemoryPool memoryPool) + => ((ReadOnlySpan)src).CopyTo(ref dst, memoryPool); /// - /// Copy to given (only payload copied to actual span/memory) + /// Unchecked Unsafe cast to a different type; for speed, it does not do the checking for "contains references" etc. that + /// does. /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CopyTo(ref SpanByteAndMemory dst, MemoryPool memoryPool) - { - if (dst.IsSpanByte) - { - if (dst.Length >= Length) - { - dst.Length = Length; - AsReadOnlySpan().CopyTo(dst.SpanByte.AsSpan()); - return; - } - dst.ConvertToHeap(); - } - - dst.Memory = memoryPool.Rent(Length); - dst.Length = Length; - AsReadOnlySpan().CopyTo(dst.Memory.Memory.Span); - } + public static ReadOnlySpan UncheckedCast(this ReadOnlySpan src) + => MemoryMarshal.CreateReadOnlySpan(ref Unsafe.As(ref MemoryMarshal.GetReference(src)), src.Length / Unsafe.SizeOf()); /// - /// Copy to given (header and payload copied to actual span/memory) + /// Copy serialized version to specified memory location /// - public void CopyWithHeaderTo(ref SpanByteAndMemory dst, MemoryPool memoryPool) + /// + /// SAFETY: The MUST point to pinned memory of at least source.TotalSize(). + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void SerializeTo(this ReadOnlySpan source, byte* destination) { - if (dst.IsSpanByte) - { - if (dst.Length >= TotalSize) - { - dst.Length = TotalSize; - var span = dst.SpanByte.AsSpan(); - fixed (byte* ptr = span) - *(int*)ptr = Length; - dst.SpanByte.ExtraMetadata = ExtraMetadata; - - AsReadOnlySpan().CopyTo(span.Slice(sizeof(int) + MetadataSize)); - return; - } - dst.ConvertToHeap(); - } - - dst.Memory = memoryPool.Rent(TotalSize); - dst.Length = TotalSize; - fixed (byte* ptr = dst.Memory.Memory.Span) - *(int*)ptr = Length; - dst.SpanByte.ExtraMetadata = ExtraMetadata; - AsReadOnlySpan().CopyTo(dst.Memory.Memory.Span.Slice(sizeof(int) + MetadataSize)); + *(int*)destination = source.Length; + source.CopyTo(new Span(destination + sizeof(int), source.Length)); } /// /// Copy serialized version to specified memory location /// + /// + /// SAFETY: The MUST point to pinned memory of at least source.TotalSize(). + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CopyTo(Span buffer) - { - fixed (byte* ptr = buffer) - CopyTo(ptr); - } + public static void SerializeTo(this Span source, byte* destination) + => ((ReadOnlySpan)source).SerializeTo(destination); /// /// Copy serialized version to specified memory location /// + /// + /// SAFETY: The MUST point to pinned memory of at least source.TotalSize(). + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CopyTo(byte* destination) + public static void SerializeTo(this ReadOnlySpan source, Span destination) { - if (Serialized) - { - *(int*)destination = length; - Buffer.MemoryCopy(Unsafe.AsPointer(ref payload), destination + sizeof(int), Length, Length); - } - else - { - *(int*)destination = length & ~UnserializedBitMask; - Buffer.MemoryCopy((void*)payload, destination + sizeof(int), Length, Length); - } + if (destination.Length < source.Length + sizeof(int)) + throw new ArgumentException($"Destination length {destination.Length} is less than source length {source.Length} + sizeof(int)"); + Unsafe.As(ref destination[0]) = source.Length; + source.CopyTo(destination.Slice(sizeof(int))); } /// - /// Gets an Etag from the payload of the SpanByte, caller should make sure the SpanByte has an Etag for the record by checking RecordInfo + /// Copy serialized version to specified memory location /// + /// + /// SAFETY: The MUST point to pinned memory of at least source.TotalSize(). + /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public long GetEtagInPayload() => *(long*)this.ToPointer(); + public static void SerializeTo(this Span source, Span destination) + => ((ReadOnlySpan)source).SerializeTo(destination); - /// - /// Gets an Etag from the payload of the SpanByte, caller should make sure the SpanByte has an Etag for the record by checking RecordInfo - /// - /// The Etag value to set + /// Length-limited string representation of a Span [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void SetEtagInPayload(long etag) => *(long*)this.ToPointer() = etag; + public static string ToShortString(this ReadOnlySpan span, int maxLen = 20) + => span.Length > maxLen ? $"{span.Slice(0, maxLen).ToString()}..." : span.ToString(); - /// - public override string ToString() - { - if (Invalid) - return "Invalid"; - var bytes = AsSpan(); - var len = Math.Min(Length, bytes.Length); - StringBuilder sb = new($"len: {Length}, mdLen: {MetadataSize}, isSer {Serialized}, "); - for (var ii = 0; ii < len; ++ii) - sb.Append(bytes[ii].ToString("x2")); - if (bytes.Length > len) - sb.Append("..."); - return sb.ToString(); - } + /// Length-limited string representation of a Span + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string ToShortString(this Span span, int maxLen = 20) + => span.Length > maxLen ? $"{span.Slice(0, maxLen).ToString()}..." : span.ToString(); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteAndMemory.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteAndMemory.cs index 6e8460c2662..b1760199c8c 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteAndMemory.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteAndMemory.cs @@ -9,14 +9,14 @@ namespace Tsavorite.core { /// - /// Output that encapsulates sync stack output (via ) and async heap output (via IMemoryOwner) + /// Output that encapsulates sync stack output (via ) and async heap output (via IMemoryOwner) /// - public unsafe struct SpanByteAndMemory + public unsafe struct SpanByteAndMemory : IDisposable { /// /// Stack output as /// - public SpanByte SpanByte; + public PinnedSpanByte SpanByte; /// /// Heap output as IMemoryOwner @@ -26,22 +26,12 @@ public unsafe struct SpanByteAndMemory /// /// Constructor using given /// - public SpanByteAndMemory(SpanByte spanByte) + public SpanByteAndMemory(PinnedSpanByte spanByte) { - if (spanByte.Serialized) throw new Exception("Cannot create new SpanByteAndMemory using serialized SpanByte"); SpanByte = spanByte; Memory = default; } - /// - /// Constructor using at given pinned , of given - /// - public SpanByteAndMemory(void* pointer, int length) - { - SpanByte = new SpanByte(length, (IntPtr)pointer); - Memory = default; - } - /// /// Get length /// @@ -54,25 +44,25 @@ public int Length /// /// Is it allocated as (on stack)? /// - public readonly bool IsSpanByte => !SpanByte.Invalid; + public readonly bool IsSpanByte => SpanByte.IsValid; /// /// Constructor using given IMemoryOwner /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] public SpanByteAndMemory(IMemoryOwner memory) { - SpanByte = default; - SpanByte.Invalid = true; + SpanByte.Invalidate(); Memory = memory; } /// /// Constructor using given IMemoryOwner and length /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] public SpanByteAndMemory(IMemoryOwner memory, int length) { - SpanByte = default; - SpanByte.Invalid = true; + SpanByte.Invalidate(); Memory = memory; SpanByte.Length = length; } @@ -80,39 +70,57 @@ public SpanByteAndMemory(IMemoryOwner memory, int length) /// /// As a span of the contained data. Use this when you haven't tested . /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ReadOnlySpan AsReadOnlySpan() => IsSpanByte ? SpanByte.AsReadOnlySpan() : Memory.Memory.Span.Slice(0, Length); + /// + /// SAFETY: This returns a null pointer in the Span if ! and is null; + /// it is the caller's responsibility to check the length and allocate if necessary. + /// + public ReadOnlySpan ReadOnlySpan + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get => IsSpanByte + ? SpanByte.ReadOnlySpan + : (Memory != null ? Memory.Memory.Span.Slice(0, Length) : new(null, 0)); + } /// - /// As a span of the contained data. Use this when you have already tested . + /// As a span of the contained data. Use this when you haven't tested . /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public readonly ReadOnlySpan AsMemoryReadOnlySpan() + /// + /// SAFETY: This returns a null pointer in the Span if ! and is null; + /// it is the caller's responsibility to check the length and allocate if necessary. + /// + public Span Span { - Debug.Assert(!IsSpanByte, "Cannot call AsMemoryReadOnlySpan when IsSpanByte"); - return Memory.Memory.Span.Slice(0, Length); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get => IsSpanByte + ? SpanByte.Span + : (Memory != null ? MemorySpan.Slice(0, Length) : new(null, 0)); } /// - /// Copy from the passed ReadOnlySpan{byte}. Use this when you have not tested . + /// As a ReadOnlySpan of the contained data. Use this when you have already tested . /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void CopyFrom(ReadOnlySpan srcSpan, MemoryPool memoryPool) + public readonly ReadOnlySpan MemoryReadOnlySpan { - if (IsSpanByte) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { - if (srcSpan.Length < Length) - { - srcSpan.CopyTo(SpanByte.AsSpan()); - Length = srcSpan.Length; - return; - } - ConvertToHeap(); + Debug.Assert(!IsSpanByte, "Cannot call MemoryReadOnlySpan when IsSpanByte"); + return Memory.Memory.Span.Slice(0, Length); } + } - Length = srcSpan.Length; - Memory = memoryPool.Rent(srcSpan.Length); - srcSpan.CopyTo(Memory.Memory.Span); + /// + /// As a Span of the contained data. Use this when you have already tested . + /// + public readonly Span MemorySpan + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get + { + Debug.Assert(!IsSpanByte, "Cannot call MemoryReadOnlySpan when IsSpanByte"); + return Memory.Memory.Span.Slice(0, Length); + } } /// @@ -121,11 +129,59 @@ public void CopyFrom(ReadOnlySpan srcSpan, MemoryPool memoryPool) /// /// SAFETY: The MUST point to pinned memory. /// - public static SpanByteAndMemory FromPinnedSpan(ReadOnlySpan span) => new(SpanByte.FromPinnedSpan(span)); + public static SpanByteAndMemory FromPinnedSpan(ReadOnlySpan span) => new() { SpanByte = PinnedSpanByte.FromPinnedSpan(span), Memory = default }; + + /// + /// Create a from a given pinned , of given + /// + /// + /// SAFETY: The MUST point to pinned memory. + /// + public static SpanByteAndMemory FromPinnedPointer(byte* pointer, int length) => new() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length), Memory = default }; /// /// Convert to be used on heap (IMemoryOwner) /// - public void ConvertToHeap() { SpanByte.Invalid = true; } + public void ConvertToHeap() { SpanByte.Invalidate(); } + + /// + /// Ensure the required size is available in this structure via the Span or the Memory. + /// + public void EnsureHeapMemorySize(int size, MemoryPool memoryPool = null) + { + if (memoryPool is null) + memoryPool = MemoryPool.Shared; + + // In case it is still SpanByte, we need to convert it to heap. This should only be done when the SpanByte is too small. + Debug.Assert(!IsSpanByte || SpanByte.Length < size, $"SpanByte Length of {SpanByte.Length} is sufficient for size of {size}, so this calling path should have used the SpanByte."); + ConvertToHeap(); + + SpanByte.Length = 0; + if (Memory is null) + { + Memory = memoryPool.Rent(size); + SpanByte.Length = size; + return; + } + + if (Memory.Memory.Length >= size) + { + SpanByte.Length = size; + return; + } + + // We have a Memory that is too small, so we need to release it and allocate a new one. + Memory.Dispose(); + Memory = null; // In case the following throws OOM + Memory = memoryPool.Rent(size); + SpanByte.Length = size; + } + + public void Dispose() + { + var memory = Memory; + Memory = null; + memory?.Dispose(); + } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteComparer.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteComparer.cs index 6ab7bbf1d1b..970bcbd9a48 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteComparer.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteComparer.cs @@ -7,9 +7,9 @@ namespace Tsavorite.core { /// - /// Equality comparer for + /// Equality comparer for /// - public struct SpanByteComparer : IKeyComparer + public struct SpanByteComparer : IKeyComparer { /// /// The default instance. @@ -18,37 +18,24 @@ public struct SpanByteComparer : IKeyComparer public static readonly SpanByteComparer Instance = new(); /// - public readonly unsafe long GetHashCode64(ref SpanByte spanByte) => StaticGetHashCode64(ref spanByte); + public readonly unsafe long GetHashCode64(ReadOnlySpan key) => StaticGetHashCode64(key); /// /// Get 64-bit hash code /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static unsafe long StaticGetHashCode64(ref SpanByte spanByte) - { - if (spanByte.Serialized) - { - byte* ptr = (byte*)Unsafe.AsPointer(ref spanByte); - return Utility.HashBytes(ptr + sizeof(int), spanByte.Length); - } - else - { - byte* ptr = (byte*)spanByte.Pointer; - return Utility.HashBytes(ptr, spanByte.Length); - } - } + public static unsafe long StaticGetHashCode64(ReadOnlySpan key) => Utility.HashBytes(key); /// - public readonly unsafe bool Equals(ref SpanByte k1, ref SpanByte k2) => StaticEquals(ref k1, ref k2); + public readonly unsafe bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => StaticEquals(k1, k2); /// /// Equality comparison /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static unsafe bool StaticEquals(ref SpanByte k1, ref SpanByte k2) + public static unsafe bool StaticEquals(ReadOnlySpan k1, ReadOnlySpan k2) { - return k1.AsReadOnlySpanWithMetadata().SequenceEqual(k2.AsReadOnlySpanWithMetadata()) - && (k1.MetadataSize == k2.MetadataSize); + return k1.SequenceEqual(k2); } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs index d9a8625b3ed..db8b2681ef6 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteFunctions.cs @@ -1,15 +1,15 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Buffers; -using System.Runtime.CompilerServices; namespace Tsavorite.core { /// - /// Callback functions for Key, Value, Input; Output; and specified + /// Callback functions for Value and Input; Output; and specified /// - public class SpanByteFunctions : SpanByteFunctions + public class SpanByteFunctions : SessionFunctionsBase { private protected readonly MemoryPool memoryPool; @@ -23,113 +23,30 @@ public SpanByteFunctions(MemoryPool memoryPool = default) } /// - public override bool SingleReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref ReadInfo readInfo) { - value.CopyTo(ref dst, memoryPool); + srcLogRecord.ValueSpan.CopyTo(ref output, memoryPool); return true; } - /// - public override bool ConcurrentReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - value.CopyTo(ref dst, memoryPool); - return true; - } + /// + public override RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = input.Length }; + /// + public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref PinnedSpanByte input) + => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + /// + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref PinnedSpanByte input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + /// + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref PinnedSpanByte input) + => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; /// - public override void ConvertOutputToHeap(ref SpanByte input, ref SpanByteAndMemory output) + public override void ConvertOutputToHeap(ref PinnedSpanByte input, ref SpanByteAndMemory output) { // Currently the default is a no-op; the derived class inspects 'input' to decide whether to ConvertToHeap(). //output.ConvertToHeap(); } } - - /// - /// Callback functions for key, value; specified , , and - /// - public class SpanByteFunctions : SessionFunctionsBase - { - /// - public override bool SingleWriter(ref SpanByte key, ref TInput input, ref SpanByte src, ref SpanByte dst, ref TOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - => DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo); - - /// - public override bool ConcurrentWriter(ref SpanByte key, ref TInput input, ref SpanByte src, ref SpanByte dst, ref TOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - => DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo); - - /// - /// Utility function for copying, Upsert version. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool DoSafeCopy(ref SpanByte src, ref SpanByte dst, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo, long metadata = 0) - { - // First get the full record length and clear it from the extra value space (if there is any). - // This ensures all bytes after the used value space are 0, which retains log-scan correctness. - - // For non-in-place operations, the new record may have been revivified, so standard copying procedure must be done; - // For SpanByte we don't implement DisposeForRevivification, so any previous value is still there, and thus we must - // zero unused value space to ensure log-scan correctness, just like in in-place updates. - - // IMPORTANT: usedValueLength and fullValueLength use .TotalSize, not .Length, to account for the leading "Length" int. - upsertInfo.ClearExtraValueLength(ref recordInfo, ref dst, dst.TotalSize); - - // We want to set the used and extra lengths and Filler whether we succeed (to the new length) or fail (to the original length). - var result = src.TrySafeCopyTo(ref dst, upsertInfo.FullValueLength, metadata); - upsertInfo.SetUsedValueLength(ref recordInfo, ref dst, dst.TotalSize); - return result; - } - - /// - /// Utility function for copying, RMW version. - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static bool DoSafeCopy(ref SpanByte src, ref SpanByte dst, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - // See comments in upsertInfo overload of this function. - rmwInfo.ClearExtraValueLength(ref recordInfo, ref dst, dst.TotalSize); - var result = src.TrySafeCopyTo(ref dst, rmwInfo.FullValueLength); - rmwInfo.SetUsedValueLength(ref recordInfo, ref dst, dst.TotalSize); - return result; - } - - /// - /// Avoids the "value = default" for added tombstone record, which do not have space for the payload - public override bool SingleDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => true; - } - - /// - /// Callback functions for key, value, input; specified and - /// - public class SpanByteFunctions : SpanByteFunctions - { - /// - public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - => DoSafeCopy(ref input, ref value, ref rmwInfo, ref recordInfo); - - /// - public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - => DoSafeCopy(ref oldValue, ref newValue, ref rmwInfo, ref recordInfo); - - /// - // The default implementation of IPU simply writes input to destination, if there is space - public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref TOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - => DoSafeCopy(ref input, ref value, ref rmwInfo, ref recordInfo); - - /// - /// Length of resulting object when doing RMW with given value and input. Here we set the length - /// to the max of input and old value lengths. You can provide a custom implementation for other cases. - /// - public override int GetRMWModifiedValueLength(ref SpanByte t, ref SpanByte input) - => sizeof(int) + (t.Length > input.Length ? t.Length : input.Length); - - /// - public override int GetRMWInitialValueLength(ref SpanByte input) => input.TotalSize; - - /// - /// Length of resulting object when doing Upsert with given value and input. Here we set the length to the - /// length of the provided value, ignoring input. You can provide a custom implementation for other cases. - /// - public override int GetUpsertValueLength(ref SpanByte t, ref SpanByte input) - => t.TotalSize; - } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs index 9220411a89c..a579ef81dba 100644 --- a/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs +++ b/libs/storage/Tsavorite/cs/src/core/VarLen/SpanByteHeapContainer.cs @@ -1,24 +1,26 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System.Runtime.CompilerServices; +using System; namespace Tsavorite.core { /// /// Heap container for structs /// - internal sealed class SpanByteHeapContainer : IHeapContainer + public sealed class SpanByteHeapContainer : IHeapContainer { readonly SectorAlignedMemory mem; + PinnedSpanByte pinnedSpanByte; - public unsafe SpanByteHeapContainer(ref SpanByte obj, SectorAlignedBufferPool pool) + public unsafe SpanByteHeapContainer(ReadOnlySpan item, SectorAlignedBufferPool pool) { - mem = pool.Get(obj.TotalSize); - obj.CopyTo(mem.GetValidPointer()); + mem = pool.Get(item.TotalSize()); + item.SerializeTo(mem.GetValidPointer()); + this.pinnedSpanByte = PinnedSpanByte.FromLengthPrefixedPinnedPointer(mem.GetValidPointer()); } - public unsafe ref SpanByte Get() => ref Unsafe.AsRef(mem.GetValidPointer()); + public unsafe ref PinnedSpanByte Get() => ref pinnedSpanByte; public void Dispose() => mem.Return(); } diff --git a/libs/storage/Tsavorite/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/libs/storage/Tsavorite/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index fb1dbb21351..c377fd1d028 100644 --- a/libs/storage/Tsavorite/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/libs/storage/Tsavorite/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -28,7 +28,7 @@ public class AzureStorageDevice : StorageDeviceBase readonly ConcurrentDictionary pendingReadWriteOperations; readonly ConcurrentDictionary pendingRemoveOperations; readonly Timer hangCheckTimer; - readonly SemaphoreSlim singleWriterSemaphore; + readonly SemaphoreSlim initialWriterSemaphore; readonly TimeSpan limit; readonly bool localBlobManager; @@ -53,7 +53,7 @@ struct RemoveRequestInfo public DateTime TimeStamp; } - SemaphoreSlim SingleWriterSemaphore => singleWriterSemaphore; + SemaphoreSlim InitialWriterSemaphore => initialWriterSemaphore; internal IStorageErrorHandler StorageErrorHandler { get; private set; } @@ -106,7 +106,7 @@ public AzureStorageDevice(string connectionString, string containerName, string StorageErrorHandler.Token.Register(CancelAllRequests); this.underLease = underLease; hangCheckTimer = new Timer(DetectHangs, null, 0, 20000); - singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; + initialWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; limit = TimeSpan.FromSeconds(90); StartAsync().Wait(); @@ -141,7 +141,7 @@ internal AzureStorageDevice(string blobName, BlobUtilsV12.BlobDirectory pageBlob StorageErrorHandler.Token.Register(CancelAllRequests); this.underLease = underLease; hangCheckTimer = new Timer(DetectHangs, null, 0, 20000); - singleWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; + initialWriterSemaphore = underLease ? new SemaphoreSlim(1) : null; limit = TimeSpan.FromSeconds(90); StartAsync().Wait(); @@ -311,7 +311,7 @@ public override void Dispose() BlobManager.StopAsync().Wait(); hangCheckTimer.Dispose(); - singleWriterSemaphore?.Dispose(); + initialWriterSemaphore?.Dispose(); // Unlike in LocalStorageDevice, we explicitly remove all page blobs if the deleteOnClose flag is set, instead of relying on the operating system // to delete files after the end of our process. This leads to potential problems if multiple instances are sharing the same underlying page blobs. @@ -675,7 +675,7 @@ unsafe void WriteToBlobAsync(BlobEntry blobEntry, IntPtr sourceAddress, ulong de if (underLease) { - SingleWriterSemaphore.Release(); + InitialWriterSemaphore.Release(); } }, TaskContinuationOptions.ExecuteSynchronously); @@ -685,7 +685,7 @@ async Task WriteToBlobAsync(BlobEntry blobEntry, IntPtr sourceAddress, long dest { if (underLease) { - await SingleWriterSemaphore.WaitAsync(); + await InitialWriterSemaphore.WaitAsync(); } long offset = 0; diff --git a/libs/storage/Tsavorite/cs/test/BasicLockTests.cs b/libs/storage/Tsavorite/cs/test/BasicLockTests.cs index 6b8139aa528..b4a1d29fe3e 100644 --- a/libs/storage/Tsavorite/cs/test/BasicLockTests.cs +++ b/libs/storage/Tsavorite/cs/test/BasicLockTests.cs @@ -12,77 +12,68 @@ namespace Tsavorite.test.LockTests { - // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - internal sealed class LocalIntKeyComparer : IKeyComparer - { - internal int mod; - - internal LocalIntKeyComparer(int mod) => this.mod = mod; - - public bool Equals(ref int k1, ref int k2) => k1 == k2; - - public long GetHashCode64(ref int k) => Utility.GetHashCode(k % mod); - } -} - -namespace Tsavorite.test.LockTests -{ - using StructStoreFunctions = StoreFunctions>; + using StructStoreFunctions = StoreFunctions; [TestFixture] public class BasicLockTests { - internal class Functions : SimpleSimpleFunctions + internal class Functions : SimpleLongSimpleFunctions { internal bool throwOnInitialUpdater; internal long initialUpdaterThrowAddress; - static bool Increment(ref int dst) + static bool Increment(Span field) { - ++dst; + ++field.AsRef(); return true; } - public override bool ConcurrentWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => Increment(ref dst); + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref long input, ReadOnlySpan srcValue, ref long output, ref UpsertInfo upsertInfo) + { + return Increment(logRecord.ValueSpan); + } - public override bool InPlaceUpdater(ref int key, ref int input, ref int value, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => Increment(ref value); + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) + { + return Increment(logRecord.ValueSpan); + } - public override bool InitialUpdater(ref int key, ref int input, ref int value, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { if (throwOnInitialUpdater) { initialUpdaterThrowAddress = rmwInfo.Address; throw new TsavoriteException(nameof(throwOnInitialUpdater)); } - return base.InitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + return base.InitialUpdater(ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); } - public override bool SingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ReadOnlySpan srcValue, ref long output, ref UpsertInfo upsertInfo) { if (throwOnInitialUpdater) { initialUpdaterThrowAddress = upsertInfo.Address; throw new TsavoriteException(nameof(throwOnInitialUpdater)); } - return base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + return base.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override bool SingleDeleter(ref int key, ref int value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public override bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { if (throwOnInitialUpdater) { initialUpdaterThrowAddress = deleteInfo.Address; throw new TsavoriteException(nameof(throwOnInitialUpdater)); } - return base.SingleDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + return base.InitialDeleter(ref logRecord, ref deleteInfo); } } - private TsavoriteKV> store; - private ClientSession> session; - private BasicContext> bContext; + private TsavoriteKV> store; + private ClientSession> session; + private BasicContext> bContext; private IDevice log; - private LocalIntKeyComparer keyComparer = new(NumRecords); + private LongKeyComparerModulo keyComparer = new(NumRecords); const int NumRecords = 100; const int ValueMult = 1000000; @@ -96,10 +87,10 @@ public void Setup() { IndexSize = 1L << 26, LogDevice = log - }, StoreFunctions.Create(keyComparer) + }, StoreFunctions.Create(keyComparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - session = store.NewSession(new Functions()); + session = store.NewSession(new Functions()); bContext = session.BasicContext; } @@ -121,10 +112,11 @@ public void TearDown() public void FunctionsLockTest([Values(1, 20)] int numThreads) { // Populate - for (var key = 0; key < NumRecords; key++) + for (long key = 0; key < NumRecords; key++) { // For this test we should be in-memory, so no pending - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + long valueNum = key * ValueMult; + ClassicAssert.IsFalse(bContext.Upsert(SpanByte.FromPinnedVariable(ref key), SpanByte.FromPinnedVariable(ref valueNum)).IsPending); } // Update @@ -133,25 +125,28 @@ public void FunctionsLockTest([Values(1, 20)] int numThreads) Task.WaitAll(tasks); // Verify - for (var key = 0; key < NumRecords; key++) + for (long key = 0; key < NumRecords; key++) { - var expectedValue = key * ValueMult + numThreads * numIters; - ClassicAssert.IsFalse(bContext.Read(key, out var value).IsPending); - ClassicAssert.AreEqual(expectedValue, value); + var expectedOutput = key * ValueMult + numThreads * numIters; + long output = 0; + ClassicAssert.IsFalse(bContext.Read(SpanByte.FromPinnedVariable(ref key), ref output).IsPending); + ClassicAssert.AreEqual(expectedOutput, output); } } void UpdateFunc(bool useRMW, int numRecords, int numIters) { - for (var key = 0; key < numRecords; ++key) + for (long keyNum = 0; keyNum < numRecords; ++keyNum) { + var key = SpanByte.FromPinnedVariable(ref keyNum); for (var iter = 0; iter < numIters; iter++) { if ((iter & 7) == 7) ClassicAssert.IsFalse(bContext.Read(key).status.IsPending); // These will both just increment the stored value, ignoring the input argument. - _ = useRMW ? bContext.RMW(key, default) : bContext.Upsert(key, default); + long input = default; + _ = useRMW ? bContext.RMW(key, ref input) : bContext.Upsert(key, SpanByte.FromPinnedVariable(ref input)); } } } @@ -161,50 +156,53 @@ void UpdateFunc(bool useRMW, int numRecords, int numIters) public unsafe void CollidingDeletedRecordTest([Values(UpdateOp.RMW, UpdateOp.Upsert)] UpdateOp updateOp, [Values(FlushMode.NoFlush, FlushMode.OnDisk)] FlushMode flushMode) { // Populate - for (var key = 0; key < NumRecords; key++) + long keyNum = 0, valueNum = 0; + for (keyNum = 0; keyNum < NumRecords; keyNum++) { // For this test we should be in-memory, so no pending - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + valueNum = keyNum * ValueMult; + ClassicAssert.IsFalse(bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref valueNum)).IsPending); } // Insert a colliding key so we don't elide the deleted key from the hash chain. - var deleteKey = NumRecords / 2; - var collidingKey = deleteKey + NumRecords; - ClassicAssert.IsFalse(bContext.Upsert(collidingKey, collidingKey * ValueMult).IsPending); + long deleteKeyNum = NumRecords / 2; + long collidingKeyNum = deleteKeyNum + NumRecords; + keyNum = collidingKeyNum; + valueNum = collidingKeyNum * ValueMult; + Span key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum), deleteKey = SpanByte.FromPinnedVariable(ref deleteKeyNum); + ClassicAssert.IsFalse(bContext.Upsert(key, value).IsPending); // Now make sure we did collide - HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(ref deleteKey)); + HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(deleteKey)); ClassicAssert.IsTrue(store.FindTag(ref hei), "Cannot find deleteKey entry"); ClassicAssert.Greater(hei.Address, Constants.kInvalidAddress, "Couldn't find deleteKey Address"); var physicalAddress = store.hlog.GetPhysicalAddress(hei.Address); - ref var recordInfo = ref store.hlog.GetInfo(physicalAddress); - ref var lookupKey = ref store.hlog.GetKey(physicalAddress); - ClassicAssert.AreEqual(collidingKey, lookupKey, "Expected collidingKey"); + var lookupKey = LogRecord.GetInlineKey(physicalAddress); + ClassicAssert.AreEqual(collidingKeyNum, lookupKey.AsRef(), "Expected collidingKey"); // Backtrace to deleteKey - physicalAddress = store.hlog.GetPhysicalAddress(recordInfo.PreviousAddress); - recordInfo = ref store.hlog.GetInfo(physicalAddress); - lookupKey = ref store.hlog.GetKey(physicalAddress); - ClassicAssert.AreEqual(deleteKey, lookupKey, "Expected deleteKey"); - ClassicAssert.IsFalse(recordInfo.Tombstone, "Tombstone should be false"); + physicalAddress = store.hlog.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"); // In-place delete. ClassicAssert.IsFalse(bContext.Delete(deleteKey).IsPending); - ClassicAssert.IsTrue(recordInfo.Tombstone, "Tombstone should be true after Delete"); + ClassicAssert.IsTrue(LogRecord.GetInfo(physicalAddress).Tombstone, "Tombstone should be true after Delete"); if (flushMode == FlushMode.ReadOnly) _ = store.hlogBase.ShiftReadOnlyAddress(store.Log.TailAddress); var status = updateOp switch { - UpdateOp.RMW => bContext.RMW(deleteKey, default), - UpdateOp.Upsert => bContext.Upsert(deleteKey, default), + UpdateOp.RMW => bContext.RMW(deleteKey, ref valueNum), + UpdateOp.Upsert => bContext.Upsert(deleteKey, value), UpdateOp.Delete => throw new InvalidOperationException("UpdateOp.Delete not expected in this test"), _ => throw new InvalidOperationException($"Unknown updateOp {updateOp}") }; ClassicAssert.IsFalse(status.IsPending); - ClassicAssert.IsTrue(recordInfo.Tombstone, "Tombstone should be true after Update"); + ClassicAssert.IsTrue(LogRecord.GetInfo(physicalAddress).Tombstone, "Tombstone should be true after Update"); } [Test] @@ -215,18 +213,19 @@ public unsafe void SetInvalidOnException([Values] UpdateOp updateOp) keyComparer.mod = int.MaxValue; // Populate - for (var key = 0; key < NumRecords; key++) + for (long keyNum = 0; keyNum < NumRecords; keyNum++) { // For this test we should be in-memory, so no pending - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + long valueNum = keyNum * ValueMult; + ClassicAssert.IsFalse(bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref valueNum)).IsPending); } var expectedThrowAddress = store.Log.TailAddress; session.functions.throwOnInitialUpdater = true; // Delete must try with an existing key; Upsert and Delete should insert a new key - var deleteKey = NumRecords / 2; - var insertKey = NumRecords + 1; + long deleteKeyNum = NumRecords / 2; + long insertKeyNum = NumRecords + 1; // Make sure everything will create a new record. store.Log.FlushAndEvict(wait: true); @@ -234,11 +233,12 @@ public unsafe void SetInvalidOnException([Values] UpdateOp updateOp) var threw = false; try { + long input = 0; var status = updateOp switch { - UpdateOp.RMW => bContext.RMW(insertKey, default), - UpdateOp.Upsert => bContext.Upsert(insertKey, default), - UpdateOp.Delete => bContext.Delete(deleteKey), + UpdateOp.RMW => bContext.RMW(SpanByte.FromPinnedVariable(ref insertKeyNum), ref input), + UpdateOp.Upsert => bContext.Upsert(SpanByte.FromPinnedVariable(ref insertKeyNum), SpanByte.FromPinnedVariable(ref input)), + UpdateOp.Delete => bContext.Delete(SpanByte.FromPinnedVariable(ref deleteKeyNum)), _ => throw new InvalidOperationException($"Unknown updateOp {updateOp}") }; ClassicAssert.IsFalse(status.IsPending); @@ -253,7 +253,7 @@ public unsafe void SetInvalidOnException([Values] UpdateOp updateOp) ClassicAssert.AreEqual(expectedThrowAddress, session.functions.initialUpdaterThrowAddress, "Unexpected throw address"); var physicalAddress = store.hlog.GetPhysicalAddress(expectedThrowAddress); - ref var recordInfo = ref store.hlog.GetInfo(physicalAddress); + var recordInfo = LogRecord.GetInfo(physicalAddress); ClassicAssert.IsTrue(recordInfo.Invalid, "Expected Invalid record"); } } diff --git a/libs/storage/Tsavorite/cs/test/BasicStorageTests.cs b/libs/storage/Tsavorite/cs/test/BasicStorageTests.cs index bf8cb9fe86d..379f43cc581 100644 --- a/libs/storage/Tsavorite/cs/test/BasicStorageTests.cs +++ b/libs/storage/Tsavorite/cs/test/BasicStorageTests.cs @@ -6,11 +6,12 @@ using NUnit.Framework.Legacy; using Tsavorite.core; using Tsavorite.devices; +using static Tsavorite.test.TestUtils; namespace Tsavorite.test { - using StructAllocator = BlittableAllocator>>; - using StructStoreFunctions = StoreFunctions>; + using StructAllocator = SpanByteAllocator>; + using StructStoreFunctions = StoreFunctions; [TestFixture] internal class BasicStorageTests @@ -19,7 +20,7 @@ internal class BasicStorageTests [Category("TsavoriteKV")] public void LocalStorageWriteRead() { - TestDeviceWriteRead(Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "BasicDiskTests.log"), deleteOnClose: true)); + TestDeviceWriteRead(Devices.CreateLogDevice(Path.Join(MethodTestDir, "BasicDiskTests.log"), deleteOnClose: true)); } [Test] @@ -27,8 +28,8 @@ public void LocalStorageWriteRead() [Category("Smoke")] public void PageBlobWriteRead() { - TestUtils.IgnoreIfNotRunningAzureTests(); - TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskTests", logger: TestUtils.TestLoggerFactory.CreateLogger("asd"))); + IgnoreIfNotRunningAzureTests(); + TestDeviceWriteRead(new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, "BasicDiskTests", logger: TestLoggerFactory.CreateLogger("asd"))); } [Test] @@ -36,8 +37,8 @@ public void PageBlobWriteRead() [Category("Smoke")] public void PageBlobWriteReadWithLease() { - TestUtils.IgnoreIfNotRunningAzureTests(); - TestDeviceWriteRead(new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskTests", null, true, true, logger: TestUtils.TestLoggerFactory.CreateLogger("asd"))); + IgnoreIfNotRunningAzureTests(); + TestDeviceWriteRead(new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, "BasicDiskTests", null, true, true, logger: TestLoggerFactory.CreateLogger("asd"))); } [Test] @@ -45,18 +46,18 @@ public void PageBlobWriteReadWithLease() [Category("Smoke")] public void TieredWriteRead() { - TestUtils.DeleteDirectory(TestUtils.MethodTestDir); + DeleteDirectory(MethodTestDir); IDevice tested; - IDevice localDevice = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "BasicDiskTests.log"), deleteOnClose: true, capacity: 1L << 30); - if (TestUtils.IsRunningAzureTests) + IDevice localDevice = Devices.CreateLogDevice(Path.Join(MethodTestDir, "BasicDiskTests.log"), deleteOnClose: true, capacity: 1L << 30); + if (IsRunningAzureTests) { - IDevice cloudDevice = new AzureStorageDevice(TestUtils.AzureEmulatedStorageString, TestUtils.AzureTestContainer, TestUtils.AzureTestDirectory, "BasicDiskTests", logger: TestUtils.TestLoggerFactory.CreateLogger("asd")); + IDevice cloudDevice = new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, "BasicDiskTests", logger: TestLoggerFactory.CreateLogger("asd")); tested = new TieredStorageDevice(1, localDevice, cloudDevice); } else { // If no Azure is enabled, just use another disk - IDevice localDevice2 = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "BasicDiskTests2.log"), deleteOnClose: true, capacity: 1L << 30); + IDevice localDevice2 = Devices.CreateLogDevice(Path.Join(MethodTestDir, "BasicDiskTests2.log"), deleteOnClose: true, capacity: 1L << 30); tested = new TieredStorageDevice(1, localDevice, localDevice2); } @@ -68,8 +69,8 @@ public void TieredWriteRead() [Category("Smoke")] public void ShardedWriteRead() { - IDevice localDevice1 = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "BasicDiskTests1.log"), deleteOnClose: true, capacity: 1L << 30); - IDevice localDevice2 = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "BasicDiskTests2.log"), deleteOnClose: true, capacity: 1L << 30); + IDevice localDevice1 = Devices.CreateLogDevice(Path.Join(MethodTestDir, "BasicDiskTests1.log"), deleteOnClose: true, capacity: 1L << 30); + IDevice localDevice2 = Devices.CreateLogDevice(Path.Join(MethodTestDir, "BasicDiskTests2.log"), deleteOnClose: true, capacity: 1L << 30); var device = new ShardedStorageDevice(new UniformPartitionScheme(512, localDevice1, localDevice2)); TestDeviceWriteRead(device); } @@ -79,11 +80,11 @@ public void ShardedWriteRead() [Category("Smoke")] public void OmitSegmentIdTest([Values] TestUtils.DeviceType deviceType) { - var filename = Path.Join(TestUtils.MethodTestDir, "test.log"); + var filename = Path.Join(MethodTestDir, "test.log"); var omit = false; for (var ii = 0; ii < 2; ++ii) { - using IDevice device = TestUtils.CreateTestDevice(deviceType, filename, omitSegmentIdFromFilename: omit); + using IDevice device = CreateTestDevice(deviceType, filename, omitSegmentIdFromFilename: omit); var storageBase = (StorageDeviceBase)device; var segmentFilename = storageBase.GetSegmentFilename(filename, 0); if (omit) @@ -96,14 +97,14 @@ public void OmitSegmentIdTest([Values] TestUtils.DeviceType deviceType) static void TestDeviceWriteRead(IDevice log) { - var store = new TsavoriteKV( + var store = new TsavoriteKV( new() { IndexSize = 1L << 26, LogDevice = log, MemorySize = 1L << 15, PageSize = 1L << 10, - }, StoreFunctions.Create(KeyStruct.Comparer.Instance) + }, StoreFunctions.Create(KeyStruct.Comparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); @@ -116,7 +117,7 @@ static void TestDeviceWriteRead(IDevice log) { var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); } _ = bContext.CompletePending(true); @@ -125,7 +126,7 @@ static void TestDeviceWriteRead(IDevice log) { var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = 1, ifield2 = 1 }; - var status = bContext.RMW(ref key1, ref input, Empty.Default); + var status = bContext.RMW(SpanByte.FromPinnedVariable(ref key1), ref input, Empty.Default); if (status.IsPending) _ = bContext.CompletePending(true); } @@ -137,7 +138,7 @@ static void TestDeviceWriteRead(IDevice log) var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (bContext.Read(ref key1, ref input, ref output, Empty.Default).IsPending) + if (bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default).IsPending) { _ = bContext.CompletePending(true); } @@ -160,7 +161,7 @@ static void TestDeviceWriteRead(IDevice log) store.Dispose(); store = null; log.Dispose(); - TestUtils.DeleteDirectory(TestUtils.MethodTestDir); + DeleteDirectory(MethodTestDir); } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/BasicTests.cs b/libs/storage/Tsavorite/cs/test/BasicTests.cs index 0988c583c92..a8118662710 100644 --- a/libs/storage/Tsavorite/cs/test/BasicTests.cs +++ b/libs/storage/Tsavorite/cs/test/BasicTests.cs @@ -12,11 +12,11 @@ namespace Tsavorite.test { - using LongAllocator = BlittableAllocator>>; - using LongStoreFunctions = StoreFunctions>; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; - using StructAllocator = BlittableAllocator>>; - using StructStoreFunctions = StoreFunctions>; + using StructAllocator = SpanByteAllocator>; + using StructStoreFunctions = StoreFunctions; //** NOTE - more detailed / in depth Read tests in ReadAddressTests.cs //** These tests ensure the basics are fully covered @@ -24,9 +24,9 @@ namespace Tsavorite.test [TestFixture] internal class BasicTests { - private TsavoriteKV store; - private ClientSession session; - private BasicContext bContext; + private TsavoriteKV store; + private ClientSession session; + private BasicContext bContext; private IDevice log; DeviceType deviceType; @@ -37,7 +37,7 @@ public void Setup() DeleteDirectory(MethodTestDir, wait: true); } - private void Setup(KVSettings kvSettings, DeviceType deviceType, int latencyMs = DefaultLocalMemoryDeviceLatencyMs) + private void Setup(KVSettings kvSettings, DeviceType deviceType, int latencyMs = DefaultLocalMemoryDeviceLatencyMs) { kvSettings.IndexSize = 1L << 13; @@ -46,7 +46,7 @@ private void Setup(KVSettings kvSettings, DeviceType dev kvSettings.LogDevice = log; store = new(kvSettings - , StoreFunctions.Create(KeyStruct.Comparer.Instance) + , StoreFunctions.Create(KeyStruct.Comparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); @@ -63,7 +63,7 @@ public void TearDown() store = null; log?.Dispose(); log = null; - DeleteDirectory(TestUtils.MethodTestDir); + DeleteDirectory(MethodTestDir); } private void AssertCompleted(Status expected, Status actual) @@ -92,8 +92,8 @@ public void NativeInMemWriteRead([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); @@ -113,27 +113,26 @@ public void NativeInMemWriteReadDelete([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); - _ = bContext.Delete(ref key1, Empty.Default); + _ = bContext.Delete(SpanByte.FromPinnedVariable(ref key1), Empty.Default); - status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.NotFound), status); var key2 = new KeyStruct { kfield1 = 14, kfield2 = 15 }; var value2 = new ValueStruct { vfield1 = 24, vfield2 = 25 }; - _ = bContext.Upsert(ref key2, ref value2, Empty.Default); - status = bContext.Read(ref key2, ref input, ref output, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key2), SpanByte.FromPinnedVariable(ref value2), Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key2), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(value2.vfield1, output.value.vfield1); ClassicAssert.AreEqual(value2.vfield2, output.value.vfield2); } - [Test] [Category("TsavoriteKV")] [Category("Smoke")] @@ -154,13 +153,13 @@ public void NativeInMemWriteReadDelete2() var key1 = new KeyStruct { kfield1 = i, kfield2 = 14 }; var value = new ValueStruct { vfield1 = i, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); } for (var i = 0; i < 10 * count; i++) { var key1 = new KeyStruct { kfield1 = i, kfield2 = 14 }; - _ = bContext.Delete(ref key1, Empty.Default); + _ = bContext.Delete(SpanByte.FromPinnedVariable(ref key1), Empty.Default); } for (var i = 0; i < 10 * count; i++) @@ -168,16 +167,16 @@ public void NativeInMemWriteReadDelete2() var key1 = new KeyStruct { kfield1 = i, kfield2 = 14 }; var value = new ValueStruct { vfield1 = i, vfield2 = 24 }; - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.NotFound), status); - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); } for (var i = 0; i < 10 * count; i++) { var key1 = new KeyStruct { kfield1 = i, kfield2 = 14 }; - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); } } @@ -203,7 +202,7 @@ public unsafe void NativeInMemWriteRead2() var i = r.Next(10000); var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable (ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); } r = new Random(10); @@ -215,7 +214,7 @@ public unsafe void NativeInMemWriteRead2() var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (bContext.Read(ref key1, ref input, ref output, Empty.Default).IsPending) + if (bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default).IsPending) { _ = bContext.CompletePending(true); } @@ -233,7 +232,7 @@ public unsafe void NativeInMemWriteRead2() var i = r.Next(10000); OutputStruct output = default; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - ClassicAssert.IsFalse(bContext.Read(ref key1, ref input, ref output, Empty.Default).Found); + ClassicAssert.IsFalse(bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default).Found); } } @@ -258,7 +257,7 @@ public unsafe void TestShiftHeadAddress([Values] DeviceType deviceType, [Values] var i = r.Next(RandRange); var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); } r = new Random(RandSeed); @@ -271,7 +270,7 @@ public unsafe void TestShiftHeadAddress([Values] DeviceType deviceType, [Values] var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (bContext.Read(ref key1, ref input, ref output, Empty.Default).IsPending) + if (bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default).IsPending) { ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); ClassicAssert.AreEqual(value.vfield2, output.value.vfield2); @@ -291,7 +290,7 @@ public unsafe void TestShiftHeadAddress([Values] DeviceType deviceType, [Values] var i = r.Next(RandRange); OutputStruct output = default; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var foundStatus = bContext.Read(ref key1, ref input, ref output, Empty.Default); + var foundStatus = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); ClassicAssert.IsTrue(foundStatus.IsPending); if (batchMode == BatchMode.NoBatch) { @@ -310,8 +309,8 @@ public unsafe void TestShiftHeadAddress([Values] DeviceType deviceType, [Values] while (outputs.Next()) { count++; - ClassicAssert.AreEqual(outputs.Current.Key.kfield1, outputs.Current.Output.value.vfield1); - ClassicAssert.AreEqual(outputs.Current.Key.kfield2, outputs.Current.Output.value.vfield2); + ClassicAssert.AreEqual(outputs.Current.Key.AsRef().kfield1, outputs.Current.Output.value.vfield1); + ClassicAssert.AreEqual(outputs.Current.Key.AsRef().kfield2, outputs.Current.Output.value.vfield2); } outputs.Dispose(); ClassicAssert.AreEqual(batchSize + (c == batchSize ? 1 : 0), count); @@ -342,14 +341,14 @@ public unsafe void NativeInMemRMWRefKeys([Values] DeviceType deviceType) var i = nums[j]; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = i, ifield2 = i + 1 }; - _ = bContext.RMW(ref key1, ref input, Empty.Default); + _ = bContext.RMW(SpanByte.FromPinnedVariable(ref key1), ref input, Empty.Default); } for (var j = 0; j < nums.Length; ++j) { var i = nums[j]; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = i, ifield2 = i + 1 }; - if (bContext.RMW(ref key1, ref input, ref output, Empty.Default).IsPending) + if (bContext.RMW(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default).IsPending) { _ = bContext.CompletePending(true); } @@ -370,7 +369,7 @@ public unsafe void NativeInMemRMWRefKeys([Values] DeviceType deviceType) key = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; ValueStruct value = new() { vfield1 = i, vfield2 = i + 1 }; - status = bContext.Read(ref key, ref input, ref output, Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(2 * value.vfield1, output.value.vfield1); @@ -378,7 +377,7 @@ public unsafe void NativeInMemRMWRefKeys([Values] DeviceType deviceType) } key = new KeyStruct { kfield1 = nums.Length, kfield2 = nums.Length + 1 }; - status = bContext.Read(ref key, ref input, ref output, Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.NotFound), status); } @@ -405,7 +404,7 @@ public unsafe void NativeInMemRMWNoRefKeys([Values] DeviceType deviceType) var i = nums[j]; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = i, ifield2 = i + 1 }; - _ = bContext.RMW(ref key1, ref input, Empty.Default); + _ = bContext.RMW(SpanByte.FromPinnedVariable(ref key1), ref input, Empty.Default); } // CopyUpdater @@ -414,7 +413,7 @@ public unsafe void NativeInMemRMWNoRefKeys([Values] DeviceType deviceType) var i = nums[j]; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = i, ifield2 = i + 1 }; - _ = bContext.RMW(key1, input); // no ref and do not set any other params + _ = bContext.RMW(SpanByte.FromPinnedVariable(ref key1), ref input); } OutputStruct output = default; @@ -428,7 +427,7 @@ public unsafe void NativeInMemRMWNoRefKeys([Values] DeviceType deviceType) key = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; ValueStruct value = new() { vfield1 = i, vfield2 = i + 1 }; - status = bContext.Read(ref key, ref input, ref output, Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(2 * value.vfield1, output.value.vfield1); @@ -436,7 +435,7 @@ public unsafe void NativeInMemRMWNoRefKeys([Values] DeviceType deviceType) } key = new KeyStruct { kfield1 = nums.Length, kfield2 = nums.Length + 1 }; - status = bContext.Read(ref key, ref input, ref output, Empty.Default); + status = bContext.Read(SpanByte.FromPinnedVariable(ref key), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.NotFound), status); } @@ -453,8 +452,9 @@ public void ReadNoRefKeyInputOutput([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - var status = bContext.Read(key1, input, out var output, Empty.Default); + OutputStruct output = default; + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); // Verify the read data @@ -474,8 +474,9 @@ public void ReadNoRefKey([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - var status = bContext.Read(key1, out var output, Empty.Default); + OutputStruct output = default; + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); // Verify the read data @@ -499,8 +500,8 @@ public void ReadWithoutInput([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - var status = bContext.Read(ref key1, ref output, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); // Verify the read data @@ -521,9 +522,9 @@ public void ReadBareMinParams([Values] DeviceType deviceType) var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); - var (status, output) = bContext.Read(key1); + var (status, output) = bContext.Read(SpanByte.FromPinnedVariable(ref key1)); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); @@ -549,7 +550,7 @@ public void ReadAtAddressDefaultOptions() var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; ReadOptions readOptions = default; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); var status = bContext.ReadAtAddress(store.Log.BeginAddress, ref input, ref output, ref readOptions, out _, Empty.Default); AssertCompleted(new(StatusCode.Found), status); @@ -563,25 +564,15 @@ class SkipReadCacheFunctions : Functions { internal long expectedReadAddress; - public override bool SingleReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) - { - Assign(ref value, ref dst, ref readInfo); - return true; - } - - public override bool ConcurrentReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - Assign(ref value, ref dst, ref readInfo); - return true; - } - - void Assign(ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) + public override bool Reader(ref TSourceLogRecord logRecord, ref InputStruct input, ref OutputStruct output, ref ReadInfo readInfo) { - dst.value = value; + output.value = logRecord.ValueSpan.AsRef(); ClassicAssert.AreEqual(expectedReadAddress, readInfo.Address); expectedReadAddress = -1; // show that the test executed + return true; } - public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) + + public override void ReadCompletionCallback(ref DiskLogRecord logRecord, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) { // Do no data verifications here; they're done in the test } @@ -608,7 +599,7 @@ public void ReadAtAddressIgnoreReadCache() var readAtAddress = store.Log.BeginAddress; Status status; - _ = skipReadCachebContext.Upsert(ref key1, ref value, Empty.Default); + _ = skipReadCachebContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value), Empty.Default); void VerifyOutput() { @@ -632,7 +623,7 @@ void VerifyResult() // This will just be an ordinary read, as the record is in memory. functions.expectedReadAddress = readAtAddress; - status = skipReadCachebContext.Read(ref key1, ref input, ref output); + status = skipReadCachebContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output); ClassicAssert.IsTrue(status.Found); VerifyOutput(); @@ -642,7 +633,7 @@ void VerifyResult() // Do not put it into the read cache. functions.expectedReadAddress = readAtAddress; ReadOptions readOptions = new() { CopyOptions = ReadCopyOptions.None }; - status = skipReadCachebContext.ReadAtAddress(readAtAddress, ref key1, ref input, ref output, ref readOptions, out _); + status = skipReadCachebContext.ReadAtAddress(readAtAddress, SpanByte.FromPinnedVariable(ref key1), ref input, ref output, ref readOptions, out _); VerifyResult(); ClassicAssert.AreEqual(store.ReadCache.BeginAddress, store.ReadCache.TailAddress); @@ -650,7 +641,7 @@ void VerifyResult() // Put it into the read cache. functions.expectedReadAddress = readAtAddress; readOptions.CopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.ReadCache); - status = skipReadCachebContext.ReadAtAddress(readAtAddress, ref key1, ref input, ref output, ref readOptions, out _); + status = skipReadCachebContext.ReadAtAddress(readAtAddress, SpanByte.FromPinnedVariable(ref key1), ref input, ref output, ref readOptions, out _); ClassicAssert.IsTrue(status.IsPending); VerifyResult(); @@ -658,7 +649,7 @@ void VerifyResult() // Now this will read from the read cache. functions.expectedReadAddress = Constants.kInvalidAddress; - status = skipReadCachebContext.Read(ref key1, ref input, ref output); + status = skipReadCachebContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output); ClassicAssert.IsFalse(status.IsPending); ClassicAssert.IsTrue(status.Found); VerifyOutput(); @@ -680,8 +671,8 @@ public void UpsertDefaultsTest([Values] DeviceType deviceType) ClassicAssert.AreEqual(0, store.EntryCount); - _ = bContext.Upsert(ref key1, ref value); - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref key1), SpanByte.FromPinnedVariable(ref value)); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, Empty.Default); AssertCompleted(new(StatusCode.Found), status); ClassicAssert.AreEqual(1, store.EntryCount); @@ -689,31 +680,6 @@ public void UpsertDefaultsTest([Values] DeviceType deviceType) ClassicAssert.AreEqual(value.vfield2, output.value.vfield2); } - // Simple Upsert test of overload where not using Ref for key and value and setting all parameters - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - public void UpsertNoRefNoDefaultsTest() - { - // Just checking more parameter values so one device is enough - deviceType = DeviceType.MLSD; - - Setup(new() { MemorySize = 1L << 29 }, deviceType); - - InputStruct input = default; - OutputStruct output = default; - - var key1 = new KeyStruct { kfield1 = 13, kfield2 = 14 }; - var value = new ValueStruct { vfield1 = 23, vfield2 = 24 }; - - _ = bContext.Upsert(key1, value, Empty.Default); - var status = bContext.Read(ref key1, ref input, ref output, Empty.Default); - AssertCompleted(new(StatusCode.Found), status); - - ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); - ClassicAssert.AreEqual(value.vfield2, output.value.vfield2); - } - //**** Quick End to End Sample code from help docs *** // Very minor changes to LogDevice call and type of Asserts to use but basically code from Sample code in docs // Also tests the overload call of .Read (ref key ref output) @@ -723,25 +689,26 @@ public static void KVBasicsSampleEndToEndInDocs() { using var log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "hlog.log"), deleteOnClose: false); - using var store = new TsavoriteKV( + using var store = new TsavoriteKV( new() { IndexSize = 1L << 26, LogDevice = log, - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s = store.NewSession>(new SimpleSimpleFunctions()); + using var s = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = s.BasicContext; - long key = 1, value = 1, input = 10, output = 0; - _ = bContext.Upsert(ref key, ref value); - _ = bContext.Read(ref key, ref output); - ClassicAssert.AreEqual(value, output); - _ = bContext.RMW(ref key, ref input); - _ = bContext.RMW(ref key, ref input); - _ = bContext.Read(ref key, ref output); + long keyNum = 1, valueNum = 1, input = 10, output = 0; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + _ = bContext.Upsert(key, value); + _ = bContext.Read(key, ref output); + ClassicAssert.AreEqual(valueNum, output); + _ = bContext.RMW(key, ref input); + _ = bContext.RMW(key, ref input); + _ = bContext.Read(key, ref output); ClassicAssert.AreEqual(10, output); } @@ -763,16 +730,16 @@ public static void BasicSyncOperationsTest() { using var log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "hlog.log"), deleteOnClose: false); - using var store = new TsavoriteKV( + using var store = new TsavoriteKV( new() { IndexSize = 1L << 26, LogDevice = log, - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; const int numRecords = 500; @@ -780,28 +747,31 @@ public static void BasicSyncOperationsTest() var hashes = new long[numRecords]; Status status; - long output; + long output = 0; - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult; - hashes[key] = store.storeFunctions.GetKeyHashCode64(ref key); + var valueNum = keyNum + valueMult; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + + hashes[keyNum] = store.storeFunctions.GetKeyHashCode64(key); status = bContext.Upsert(key, value); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); - status = bContext.Read(key, out output); + status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } void doUpdate(bool useRMW) { // Update and Read without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult * 2; + var valueNum = keyNum + valueMult * 2; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); if (useRMW) { - status = bContext.RMW(key, value); + status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } else @@ -809,31 +779,32 @@ void doUpdate(bool useRMW) status = bContext.Upsert(key, value); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } - status = bContext.Read(key, out output); + status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } // Update and Read with keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult * 3; + var valueNum = keyNum + valueMult * 3; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); if (useRMW) { - RMWOptions rmwOptions = new() { KeyHash = hashes[key] }; - status = bContext.RMW(key, value, ref rmwOptions); + RMWOptions rmwOptions = new() { KeyHash = hashes[keyNum] }; + status = bContext.RMW(key, ref valueNum, ref rmwOptions); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } else { - UpsertOptions upsertOptions = new() { KeyHash = hashes[key] }; + UpsertOptions upsertOptions = new() { KeyHash = hashes[keyNum] }; status = bContext.Upsert(key, value, ref upsertOptions); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } - ReadOptions readOptions = new() { KeyHash = hashes[key] }; - status = bContext.Read(key, out output, ref readOptions); + ReadOptions readOptions = new() { KeyHash = hashes[keyNum] }; + status = bContext.Read(key, ref output, ref readOptions); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } } @@ -841,21 +812,23 @@ void doUpdate(bool useRMW) doUpdate(useRMW: true); // Delete without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { + var key = SpanByte.FromPinnedVariable(ref keyNum); status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); - status = bContext.Read(key, out _); + status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.NotFound, status.ToString()); } // Update and Read without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - DeleteOptions deleteOptions = new() { KeyHash = hashes[key] }; + var key = SpanByte.FromPinnedVariable(ref keyNum); + DeleteOptions deleteOptions = new() { KeyHash = hashes[keyNum] }; status = bContext.Delete(key, ref deleteOptions); - ReadOptions readOptions = new() { KeyHash = hashes[key] }; - status = bContext.Read(key, out _, ref readOptions); + ReadOptions readOptions = new() { KeyHash = hashes[keyNum] }; + status = bContext.Read(key, ref output, ref readOptions); ClassicAssert.IsTrue(status.NotFound, status.ToString()); } } @@ -866,16 +839,16 @@ public static void BasicOperationsTest() { using var log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "hlog.log"), deleteOnClose: false); - using var store = new TsavoriteKV( + using var store = new TsavoriteKV( new() { IndexSize = 1L << 26, LogDevice = log, - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; const int numRecords = 500; @@ -885,26 +858,30 @@ public static void BasicOperationsTest() Status status; long output; - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult; - hashes[key] = store.storeFunctions.GetKeyHashCode64(ref key); + var valueNum = keyNum + valueMult; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + + hashes[keyNum] = store.storeFunctions.GetKeyHashCode64(key); status = bContext.Upsert(key, value); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); (status, output) = bContext.Read(key); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } void doUpdate(bool useRMW) { // Update and Read without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult * 2; + var valueNum = keyNum + valueMult * 2; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + if (useRMW) { - status = bContext.RMW(key, value); + status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } else @@ -914,29 +891,31 @@ void doUpdate(bool useRMW) } (status, output) = bContext.Read(key); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } // Update and Read with keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - var value = key + valueMult * 3; + var valueNum = keyNum + valueMult * 3; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + if (useRMW) { - RMWOptions rmwOptions = new() { KeyHash = hashes[key] }; - status = bContext.RMW(key, value, ref rmwOptions); + RMWOptions rmwOptions = new() { KeyHash = hashes[keyNum] }; + status = bContext.RMW(key, ref valueNum, ref rmwOptions); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } else { - UpsertOptions upsertOptions = new() { KeyHash = hashes[key] }; + UpsertOptions upsertOptions = new() { KeyHash = hashes[keyNum] }; status = bContext.Upsert(key, value, ref upsertOptions); ClassicAssert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); } - ReadOptions readOptions = new() { KeyHash = hashes[key] }; + ReadOptions readOptions = new() { KeyHash = hashes[keyNum] }; (status, output) = bContext.Read(key, ref readOptions); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(value, output); + ClassicAssert.AreEqual(valueNum, output); } } @@ -944,8 +923,10 @@ void doUpdate(bool useRMW) doUpdate(useRMW: true); // Delete without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { + var key = SpanByte.FromPinnedVariable(ref keyNum); + status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); (status, _) = bContext.Read(key); @@ -953,11 +934,13 @@ void doUpdate(bool useRMW) } // Update and Read without keyHash - for (var key = 0L; key < numRecords; key++) + for (var keyNum = 0L; keyNum < numRecords; keyNum++) { - DeleteOptions deleteOptions = new() { KeyHash = hashes[key] }; + var key = SpanByte.FromPinnedVariable(ref keyNum); + + DeleteOptions deleteOptions = new() { KeyHash = hashes[keyNum] }; status = bContext.Delete(key, ref deleteOptions); - ReadOptions readOptions = new() { KeyHash = hashes[key] }; + ReadOptions readOptions = new() { KeyHash = hashes[keyNum] }; (status, _) = bContext.Read(key, ref readOptions); ClassicAssert.IsTrue(status.NotFound, status.ToString()); } diff --git a/libs/storage/Tsavorite/cs/test/BlittableIterationTests.cs b/libs/storage/Tsavorite/cs/test/BlittableIterationTests.cs deleted file mode 100644 index d12d6dd1acf..00000000000 --- a/libs/storage/Tsavorite/cs/test/BlittableIterationTests.cs +++ /dev/null @@ -1,278 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.IO; -using System.Threading.Tasks; -using NUnit.Framework; -using NUnit.Framework.Legacy; -using Tsavorite.core; -using static Tsavorite.test.TestUtils; - -namespace Tsavorite.test -{ - using StructStoreFunctions = StoreFunctions>; - - [TestFixture] - internal class BlittableIterationTests - { - private TsavoriteKV> store; - private IDevice log; - - [SetUp] - public void Setup() - { - // Clean up log files from previous test runs in case they weren't cleaned up - DeleteDirectory(MethodTestDir, wait: true); - } - - [TearDown] - public void TearDown() - { - store?.Dispose(); - store = null; - log?.Dispose(); - log = null; - DeleteDirectory(MethodTestDir); - } - - internal struct BlittablePushIterationTestFunctions : IScanIteratorFunctions - { - internal int keyMultToValue; - internal long numRecords; - internal int stopAt; - - public bool SingleReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - { - cursorRecordResult = CursorRecordResult.Accept; // default; not used here - if (keyMultToValue > 0) - ClassicAssert.AreEqual(key.kfield1 * keyMultToValue, value.vfield1); - return stopAt != ++numRecords; - } - - public bool ConcurrentReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - public readonly bool OnStart(long beginAddress, long endAddress) => true; - public readonly void OnException(Exception exception, long numberOfRecords) { } - public readonly void OnStop(bool completed, long numberOfRecords) { } - } - - [Test] - [Category(TsavoriteKVTestCategory)] - [Category(SmokeTestCategory)] - public void BlittableIterationBasicTest([Values] DeviceType deviceType, [Values] ScanIteratorType scanIteratorType) - { - log = CreateTestDevice(deviceType, Path.Join(MethodTestDir, $"{deviceType}.log")); - - store = new( - new() - { - IndexSize = 1L << 26, - LogDevice = log, - MemorySize = 1L << 15, - PageSize = 1L << 9, - SegmentSize = 1L << 22 - }, StoreFunctions.Create(KeyStruct.Comparer.Instance) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - - using var session = store.NewSession(new FunctionsCompaction()); - var bContext = session.BasicContext; - - BlittablePushIterationTestFunctions scanIteratorFunctions = new(); - - const int totalRecords = 500; - - void iterateAndVerify(int keyMultToValue, int expectedRecs) - { - scanIteratorFunctions.keyMultToValue = keyMultToValue; - scanIteratorFunctions.numRecords = 0; - - if (scanIteratorType == ScanIteratorType.Pull) - { - using var iter = session.Iterate(); - while (iter.GetNext(out var recordInfo)) - _ = scanIteratorFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); - } - else - ClassicAssert.IsTrue(session.Iterate(ref scanIteratorFunctions), $"Failed to complete push iteration; numRecords = {scanIteratorFunctions.numRecords}"); - - ClassicAssert.AreEqual(expectedRecs, scanIteratorFunctions.numRecords); - } - - // Initial population - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - iterateAndVerify(1, totalRecords); - - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = 2 * i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - iterateAndVerify(2, totalRecords); - - for (var i = totalRecords / 2; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - iterateAndVerify(0, totalRecords); - - for (var i = 0; i < totalRecords; i += 2) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - iterateAndVerify(0, totalRecords); - - for (var i = 0; i < totalRecords; i += 2) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - _ = bContext.Delete(ref key1); - } - iterateAndVerify(0, totalRecords / 2); - - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = 3 * i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - iterateAndVerify(3, totalRecords); - - store.Log.FlushAndEvict(wait: true); - iterateAndVerify(3, totalRecords); - } - - [Test] - [Category(TsavoriteKVTestCategory)] - [Category(SmokeTestCategory)] - public void BlittableIterationPushStopTest() - { - log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "stop_test.log")); - - store = new( - new() - { - IndexSize = 1L << 26, - LogDevice = log, - MemorySize = 1L << 15, - PageSize = 1L << 9, - SegmentSize = 1L << 22 - }, StoreFunctions.Create(KeyStruct.Comparer.Instance) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - - using var session = store.NewSession(new FunctionsCompaction()); - var bContext = session.BasicContext; - BlittablePushIterationTestFunctions scanIteratorFunctions = new(); - - const int totalRecords = 2000; - var start = store.Log.TailAddress; - - void scanAndVerify(int stopAt, bool useScan) - { - scanIteratorFunctions.numRecords = 0; - scanIteratorFunctions.stopAt = stopAt; - if (useScan) - ClassicAssert.IsFalse(store.Log.Scan(ref scanIteratorFunctions, start, store.Log.TailAddress), $"Failed to terminate push iteration early; numRecords = {scanIteratorFunctions.numRecords}"); - else - ClassicAssert.IsFalse(session.Iterate(ref scanIteratorFunctions), $"Failed to terminate push iteration early; numRecords = {scanIteratorFunctions.numRecords}"); - ClassicAssert.AreEqual(stopAt, scanIteratorFunctions.numRecords); - } - - // Initial population - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - - scanAndVerify(42, useScan: true); - scanAndVerify(42, useScan: false); - } - - [Test] - [Category(TsavoriteKVTestCategory)] - [Category(SmokeTestCategory)] - public unsafe void BlittableIterationPushLockTest([Values(1, 4)] int scanThreads, [Values(1, 4)] int updateThreads, [Values] ScanMode scanMode) - { - log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "lock_test.log")); - - // Must be large enough to contain all records in memory to exercise locking - store = new( - new() - { - IndexSize = 1L << 26, - LogDevice = log, - MemorySize = 1L << 25, - PageSize = 1L << 20, - SegmentSize = 1L << 22 - }, StoreFunctions.Create(KeyStruct.Comparer.Instance) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - - const int totalRecords = 2000; - var start = store.Log.TailAddress; - - void LocalScan(int i) - { - using var session = store.NewSession(new FunctionsCompaction()); - BlittablePushIterationTestFunctions scanIteratorFunctions = new(); - if (scanMode == ScanMode.Scan) - ClassicAssert.IsTrue(store.Log.Scan(ref scanIteratorFunctions, start, store.Log.TailAddress), $"Failed to complete push scan; numRecords = {scanIteratorFunctions.numRecords}"); - else - ClassicAssert.IsTrue(session.Iterate(ref scanIteratorFunctions), $"Failed to complete push iteration; numRecords = {scanIteratorFunctions.numRecords}"); - ClassicAssert.AreEqual(totalRecords, scanIteratorFunctions.numRecords); - } - - void LocalUpdate(int tid) - { - using var session = store.NewSession(new FunctionsCompaction()); - var bContext = session.BasicContext; - for (var iteration = 0; iteration < 2; ++iteration) - { - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = (tid + 1) * i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); - } - } - } - - { // Initial population - using var session = store.NewSession(new FunctionsCompaction()); - var bContext = session.BasicContext; - for (var i = 0; i < totalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - } - - List tasks = []; // Task rather than Thread for propagation of exception. - var numThreads = scanThreads + updateThreads; - for (var t = 0; t < numThreads; t++) - { - var tid = t; - if (t < scanThreads) - tasks.Add(Task.Factory.StartNew(() => LocalScan(tid))); - else - tasks.Add(Task.Factory.StartNew(() => LocalUpdate(tid))); - } - Task.WaitAll([.. tasks]); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/BlittableLogScanTests.cs b/libs/storage/Tsavorite/cs/test/BlittableLogScanTests.cs deleted file mode 100644 index bb94d583eb5..00000000000 --- a/libs/storage/Tsavorite/cs/test/BlittableLogScanTests.cs +++ /dev/null @@ -1,396 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.IO; -using NUnit.Framework; -using NUnit.Framework.Legacy; -using Tsavorite.core; -using static Tsavorite.test.TestUtils; - -namespace Tsavorite.test -{ - // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - struct KeyStructComparerModulo : IKeyComparer - { - readonly long mod; - - internal KeyStructComparerModulo(long mod) => this.mod = mod; - - public readonly bool Equals(ref KeyStruct k1, ref KeyStruct k2) => k1.kfield1 == k2.kfield1 && k1.kfield2 == k2.kfield2; - - // Force collisions to create a chain - public readonly long GetHashCode64(ref KeyStruct key) - { - long hash = Utility.GetHashCode(key.kfield1); - return mod > 0 ? hash % mod : hash; - } - } -} - -namespace Tsavorite.test -{ - using StructAllocator = BlittableAllocator>>; - using StructStoreFunctions = StoreFunctions>; - - [TestFixture] - internal class BlittableLogScanTests - { - private TsavoriteKV store; - private IDevice log; - const int TotalRecords = 2000; - const int PageSizeBits = 10; - - [SetUp] - public void Setup() - { - DeleteDirectory(MethodTestDir, wait: true); - - KeyStructComparerModulo comparer = new(0); - foreach (var arg in TestContext.CurrentContext.Test.Arguments) - { - if (arg is HashModulo mod && mod == HashModulo.Hundred) - { - comparer = new(100); - continue; - } - } - - log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "test.log"), deleteOnClose: true); - store = new(new() - { - IndexSize = 1L << 26, - LogDevice = log, - MemorySize = 1L << 24, - PageSize = 1L << PageSizeBits - }, StoreFunctions.Create(comparer) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - } - - [TearDown] - public void TearDown() - { - store?.Dispose(); - store = null; - log?.Dispose(); - log = null; - DeleteDirectory(MethodTestDir); - } - - internal struct BlittablePushScanTestFunctions : IScanIteratorFunctions - { - internal long numRecords; - - public readonly bool OnStart(long beginAddress, long endAddress) => true; - - public bool ConcurrentReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - - public bool SingleReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - { - cursorRecordResult = CursorRecordResult.Accept; // default; not used here - ClassicAssert.AreEqual(numRecords, key.kfield1); - ClassicAssert.AreEqual(numRecords + 1, key.kfield2); - ClassicAssert.AreEqual(numRecords, value.vfield1); - ClassicAssert.AreEqual(numRecords + 1, value.vfield2); - - ++numRecords; - return true; - } - - public readonly void OnException(Exception exception, long numberOfRecords) { } - - public readonly void OnStop(bool completed, long numberOfRecords) { } - } - - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - - public void BlittableDiskWriteScan([Values] ScanIteratorType scanIteratorType) - { - using var session = store.NewSession(new Functions()); - var bContext = session.BasicContext; - - using var s = store.Log.Subscribe(new LogObserver()); - var start = store.Log.TailAddress; - - for (int i = 0; i < TotalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - } - store.Log.FlushAndEvict(true); - - BlittablePushScanTestFunctions scanIteratorFunctions = new(); - void scanAndVerify(ScanBufferingMode sbm) - { - scanIteratorFunctions.numRecords = 0; - - if (scanIteratorType == ScanIteratorType.Pull) - { - using var iter = store.Log.Scan(start, store.Log.TailAddress, sbm); - while (iter.GetNext(out var recordInfo)) - _ = scanIteratorFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); - } - else - ClassicAssert.IsTrue(store.Log.Scan(ref scanIteratorFunctions, start, store.Log.TailAddress, sbm), "Failed to complete push iteration"); - - ClassicAssert.AreEqual(TotalRecords, scanIteratorFunctions.numRecords); - } - - scanAndVerify(ScanBufferingMode.SinglePageBuffering); - scanAndVerify(ScanBufferingMode.DoublePageBuffering); - } - - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - - public void BlittableScanJumpToBeginAddressTest() - { - using var session = store.NewSession(new Functions()); - var bContext = session.BasicContext; - - const int numRecords = 200; - const int numTailRecords = 10; - long shiftBeginAddressTo = 0; - int shiftToKey = 0; - for (int i = 0; i < numRecords; i++) - { - if (i == numRecords - numTailRecords) - { - shiftBeginAddressTo = store.Log.TailAddress; - shiftToKey = i; - } - var key = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key, ref value, Empty.Default); - } - - using var iter = store.Log.Scan(store.Log.HeadAddress, store.Log.TailAddress); - - for (int i = 0; i < 100; ++i) - { - ClassicAssert.IsTrue(iter.GetNext(out var recordInfo)); - ClassicAssert.AreEqual(i, iter.GetKey().kfield1); - ClassicAssert.AreEqual(i, iter.GetValue().vfield1); - } - - store.Log.ShiftBeginAddress(shiftBeginAddressTo); - - for (int i = 0; i < numTailRecords; ++i) - { - ClassicAssert.IsTrue(iter.GetNext(out var recordInfo)); - if (i == 0) - ClassicAssert.AreEqual(store.Log.BeginAddress, iter.CurrentAddress); - var expectedKey = numRecords - numTailRecords + i; - ClassicAssert.AreEqual(expectedKey, iter.GetKey().kfield1); - ClassicAssert.AreEqual(expectedKey, iter.GetValue().vfield1); - } - } - - public class ScanFunctions : FunctionsWithContext - { - // Right now this is unused but helped with debugging so I'm keeping it around. - internal long insertedAddress; - - public override bool SingleWriter(ref KeyStruct key, ref InputStruct input, ref ValueStruct src, ref ValueStruct dst, ref OutputStruct output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - insertedAddress = upsertInfo.Address; - return base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); - } - } - - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - - public void BlittableScanCursorTest([Values(HashModulo.NoMod, HashModulo.Hundred)] HashModulo hashMod) - { - const long PageSize = 1L << PageSizeBits; - var recordSize = BlittableAllocatorImpl.RecordSize; - - using var session = store.NewSession(new ScanFunctions()); - var bContext = session.BasicContext; - - for (int i = 0; i < TotalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - - var scanCursorFuncs = new ScanCursorFuncs(); - - // Normal operations - var endAddresses = new long[] { store.Log.TailAddress, long.MaxValue }; - var counts = new long[] { 10, 100, long.MaxValue }; - - long cursor = 0; - for (var iAddr = 0; iAddr < endAddresses.Length; ++iAddr) - { - for (var iCount = 0; iCount < counts.Length; ++iCount) - { - scanCursorFuncs.Initialize(verifyKeys: true); - while (session.ScanCursor(ref cursor, counts[iCount], scanCursorFuncs, endAddresses[iAddr])) - ; - ClassicAssert.AreEqual(TotalRecords, scanCursorFuncs.numRecords, $"count: {counts[iCount]}, endAddress {endAddresses[iAddr]}"); - ClassicAssert.AreEqual(0, cursor, "Expected cursor to be 0, pt 1"); - } - } - - // After FlushAndEvict, we will be doing pending IO. With collision chains, this means we may be returning colliding keys from in-memory - // before the sequential keys from pending IO. Therefore we do not want to verify keys if we are causing collisions. - store.Log.FlushAndEvict(wait: true); - bool verifyKeys = hashMod == HashModulo.NoMod; - - // Scan and verify we see them all - scanCursorFuncs.Initialize(verifyKeys); - ClassicAssert.IsFalse(session.ScanCursor(ref cursor, long.MaxValue, scanCursorFuncs, long.MaxValue), "Expected scan to finish and return false, pt 1"); - ClassicAssert.AreEqual(TotalRecords, scanCursorFuncs.numRecords, "Unexpected count for all on-disk"); - ClassicAssert.AreEqual(0, cursor, "Expected cursor to be 0, pt 2"); - - // Add another totalRecords, with keys incremented by totalRecords to remain distinct, and verify we see all keys. - for (int i = 0; i < TotalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i + TotalRecords, kfield2 = i + TotalRecords + 1 }; - var value = new ValueStruct { vfield1 = i + TotalRecords, vfield2 = i + TotalRecords + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - scanCursorFuncs.Initialize(verifyKeys); - ClassicAssert.IsFalse(session.ScanCursor(ref cursor, long.MaxValue, scanCursorFuncs, long.MaxValue), "Expected scan to finish and return false, pt 1"); - ClassicAssert.AreEqual(TotalRecords * 2, scanCursorFuncs.numRecords, "Unexpected count for on-disk + in-mem"); - ClassicAssert.AreEqual(0, cursor, "Expected cursor to be 0, pt 3"); - - // Try an invalid cursor (not a multiple of 8) on-disk and verify we get one correct record. Use 3x page size to make sure page boundaries are tested. - ClassicAssert.Greater(store.hlogBase.GetTailAddress(), PageSize * 10, "Need enough space to exercise this"); - scanCursorFuncs.Initialize(verifyKeys); - cursor = store.hlogBase.BeginAddress - 1; - do - { - ClassicAssert.IsTrue(session.ScanCursor(ref cursor, 1, scanCursorFuncs, long.MaxValue, validateCursor: true), "Expected scan to finish and return false, pt 1"); - cursor = scanCursorFuncs.lastAddress + recordSize + 1; - } while (cursor < PageSize * 3); - - // Now try an invalid cursor in-memory. First we have to read what's at the target start address (let's use HeadAddress) to find what the value is. - InputStruct input = default; - OutputStruct output = default; - ReadOptions readOptions = default; - var readStatus = bContext.ReadAtAddress(store.hlogBase.HeadAddress, ref input, ref output, ref readOptions, out _); - ClassicAssert.IsTrue(readStatus.Found, $"Could not read at HeadAddress; {readStatus}"); - - scanCursorFuncs.Initialize(verifyKeys); - scanCursorFuncs.numRecords = (int)output.value.vfield1; - cursor = store.Log.HeadAddress + 1; - do - { - ClassicAssert.IsTrue(session.ScanCursor(ref cursor, 1, scanCursorFuncs, long.MaxValue, validateCursor: true), "Expected scan to finish and return false, pt 1"); - cursor = scanCursorFuncs.lastAddress + recordSize + 1; - } while (cursor < store.hlogBase.HeadAddress + PageSize * 3); - } - - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - - public void BlittableScanCursorFilterTest([Values(HashModulo.NoMod, HashModulo.Hundred)] HashModulo hashMod) - { - var recordSize = BlittableAllocatorImpl.RecordSize; - - using var session = store.NewSession(new ScanFunctions()); - var bContext = session.BasicContext; - - for (int i = 0; i < TotalRecords; i++) - { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value); - } - - var scanCursorFuncs = new ScanCursorFuncs(); - - long cursor = 0; - scanCursorFuncs.Initialize(verifyKeys: false, k => k.kfield1 % 10 == 0); - ClassicAssert.IsTrue(session.ScanCursor(ref cursor, 10, scanCursorFuncs, store.Log.TailAddress), "ScanCursor failed, pt 1"); - ClassicAssert.AreEqual(10, scanCursorFuncs.numRecords, "count at first 10"); - ClassicAssert.Greater(cursor, 0, "Expected cursor to be > 0, pt 1"); - - // Now fake out the key verification to make it think we got all the previous keys; this ensures we are aligned as expected. - scanCursorFuncs.Initialize(verifyKeys: true, k => true); - scanCursorFuncs.numRecords = 91; // (filter accepts: 0-9) * 10 + 1 - ClassicAssert.IsTrue(session.ScanCursor(ref cursor, 100, scanCursorFuncs, store.Log.TailAddress), "ScanCursor failed, pt 2"); - ClassicAssert.AreEqual(191, scanCursorFuncs.numRecords, "count at second 100"); - ClassicAssert.Greater(cursor, 0, "Expected cursor to be > 0, pt 1"); - } - - internal sealed class ScanCursorFuncs : IScanIteratorFunctions - { - internal int numRecords; - internal long lastAddress; - internal bool verifyKeys; - internal Func filter; - - internal void Initialize(bool verifyKeys) => Initialize(verifyKeys, k => true); - - internal void Initialize(bool verifyKeys, Func filter) - { - numRecords = 0; - this.verifyKeys = verifyKeys; - this.filter = filter; - } - - public bool ConcurrentReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - { - cursorRecordResult = filter(key) ? CursorRecordResult.Accept : CursorRecordResult.Skip; - if (cursorRecordResult != CursorRecordResult.Accept) - return true; - - if (verifyKeys) - ClassicAssert.AreEqual(numRecords, key.kfield1, "Mismatched key field on Scan"); - ClassicAssert.Greater(recordMetadata.Address, 0); - ++numRecords; - lastAddress = recordMetadata.Address; - return true; - } - - public void OnException(Exception exception, long numberOfRecords) - => Assert.Fail($"Unexpected exception at {numberOfRecords} records: {exception.Message}"); - - public bool OnStart(long beginAddress, long endAddress) => true; - - public void OnStop(bool completed, long numberOfRecords) { } - - public bool SingleReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => ConcurrentReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - } - - class LogObserver : IObserver> - { - int val = 0; - - public void OnCompleted() - { - ClassicAssert.AreEqual(TotalRecords, val); - } - - public void OnError(Exception error) - { - } - - public void OnNext(ITsavoriteScanIterator iter) - { - while (iter.GetNext(out _, out KeyStruct key, out ValueStruct value)) - { - ClassicAssert.AreEqual(val, key.kfield1); - ClassicAssert.AreEqual(val + 1, key.kfield2); - ClassicAssert.AreEqual(val, value.vfield1); - ClassicAssert.AreEqual(val + 1, value.vfield2); - val++; - } - } - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/CancellationTests.cs b/libs/storage/Tsavorite/cs/test/CancellationTests.cs index 9d85f899c93..dcfb48d710e 100644 --- a/libs/storage/Tsavorite/cs/test/CancellationTests.cs +++ b/libs/storage/Tsavorite/cs/test/CancellationTests.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -9,8 +10,9 @@ namespace Tsavorite.test.Cancellation { - using IntAllocator = BlittableAllocator>>; - using IntStoreFunctions = StoreFunctions>; + // Use an int in these tests just to get a different length underlying the SpanByte + using IntAllocator = SpanByteAllocator>; + using IntStoreFunctions = StoreFunctions; [TestFixture] class CancellationTests @@ -23,16 +25,16 @@ internal enum CancelLocation NeedCopyUpdate, CopyUpdater, InPlaceUpdater, - SingleWriter, - ConcurrentWriter + InitialWriter, + InPlaceWriter } - public class CancellationFunctions : SessionFunctionsBase + public class CancellationFunctions : SessionFunctionsBase { internal CancelLocation cancelLocation = CancelLocation.None; internal CancelLocation lastFunc = CancelLocation.None; - public override bool NeedInitialUpdate(ref int key, ref int input, ref int output, ref RMWInfo rmwInfo) + public override bool NeedInitialUpdate(ReadOnlySpan key, ref int input, ref int output, ref RMWInfo rmwInfo) { lastFunc = CancelLocation.NeedInitialUpdate; if (cancelLocation == CancelLocation.NeedInitialUpdate) @@ -43,7 +45,7 @@ public override bool NeedInitialUpdate(ref int key, ref int input, ref int outpu return true; } - public override bool NeedCopyUpdate(ref int key, ref int input, ref int oldValue, ref int output, ref RMWInfo rmwInfo) + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref int input, ref int output, ref RMWInfo rmwInfo) { lastFunc = CancelLocation.NeedCopyUpdate; if (cancelLocation == CancelLocation.NeedCopyUpdate) @@ -55,7 +57,7 @@ public override bool NeedCopyUpdate(ref int key, ref int input, ref int oldValue } /// - public override bool CopyUpdater(ref int key, ref int input, ref int oldValue, ref int newValue, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref int input, ref int output, ref RMWInfo rmwInfo) { lastFunc = CancelLocation.CopyUpdater; ClassicAssert.AreNotEqual(CancelLocation.NeedCopyUpdate, cancelLocation); @@ -64,11 +66,10 @@ public override bool CopyUpdater(ref int key, ref int input, ref int oldValue, r rmwInfo.Action = RMWAction.CancelOperation; return false; } - newValue = oldValue; - return true; + return dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo); } - public override bool InitialUpdater(ref int key, ref int input, ref int value, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref int input, ref int output, ref RMWInfo rmwInfo) { lastFunc = CancelLocation.InitialUpdater; ClassicAssert.AreNotEqual(CancelLocation.NeedInitialUpdate, cancelLocation); @@ -78,11 +79,10 @@ public override bool InitialUpdater(ref int key, ref int input, ref int value, r rmwInfo.Action = RMWAction.CancelOperation; return false; } - value = input; - return true; + return logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref input), ref sizeInfo); } - public override bool InPlaceUpdater(ref int key, ref int input, ref int value, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref int input, ref int output, ref RMWInfo rmwInfo) { lastFunc = CancelLocation.InPlaceUpdater; if (cancelLocation == CancelLocation.InPlaceUpdater) @@ -90,41 +90,51 @@ public override bool InPlaceUpdater(ref int key, ref int input, ref int value, r rmwInfo.Action = RMWAction.CancelOperation; return false; } - value = input; - return true; + return logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref input), ref sizeInfo); } // Upsert functions - public override bool SingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref int input, ReadOnlySpan srcValue, ref int output, ref UpsertInfo upsertInfo) { - lastFunc = CancelLocation.SingleWriter; - if (cancelLocation == CancelLocation.SingleWriter) + lastFunc = CancelLocation.InitialWriter; + if (cancelLocation == CancelLocation.InitialWriter) { upsertInfo.Action = UpsertAction.CancelOperation; return false; } - dst = src; - return true; + return logRecord.TrySetValueSpan(srcValue, ref sizeInfo); } - public override bool ConcurrentWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref int input, ReadOnlySpan srcValue, ref int output, ref UpsertInfo upsertInfo) { - lastFunc = CancelLocation.ConcurrentWriter; - if (cancelLocation == CancelLocation.ConcurrentWriter) + lastFunc = CancelLocation.InPlaceWriter; + if (cancelLocation == CancelLocation.InPlaceWriter) { upsertInfo.Action = UpsertAction.CancelOperation; return false; } - dst = src; - return true; + return logRecord.TrySetValueSpan(srcValue, ref sizeInfo); } + + /// + public override RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref int input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(int) }; + /// + public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref int input) + => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(int) }; + /// + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref int input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; + /// + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref int input) + => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } IDevice log; CancellationFunctions functions; - TsavoriteKV store; - ClientSession session; - BasicContext bContext; + TsavoriteKV store; + ClientSession session; + BasicContext bContext; const int NumRecs = 100; @@ -140,7 +150,7 @@ public void Setup() LogDevice = log, MemorySize = 1L << 17, PageSize = 1L << 12 - }, StoreFunctions.Create(IntKeyComparer.Instance) + }, StoreFunctions.Create(IntKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); @@ -164,8 +174,11 @@ public void TearDown() private unsafe void Populate() { // Single alloc outside the loop, to the max length we'll need. - for (int ii = 0; ii < NumRecs; ii++) - _ = bContext.Upsert(ii, ii * NumRecs * 10); + for (int keyNum = 0; keyNum < NumRecs; keyNum++) + { + var valueNum = keyNum * NumRecs * 10; + _ = bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref valueNum)); + } } [Test] @@ -175,15 +188,17 @@ public void InitialUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - int key = NumRecs; + int keyNum = NumRecs, valueNum = keyNum * NumRecs * 10; + var key = SpanByte.FromPinnedVariable(ref keyNum); functions.cancelLocation = CancelLocation.NeedInitialUpdate; - var status = bContext.RMW(key, key * NumRecs * 10); + var status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.IsCanceled); ClassicAssert.AreEqual(CancelLocation.NeedInitialUpdate, functions.lastFunc); functions.cancelLocation = CancelLocation.InitialUpdater; - status = bContext.RMW(key, key * NumRecs * 10); + valueNum *= 2; + status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.IsCanceled); ClassicAssert.AreEqual(CancelLocation.InitialUpdater, functions.lastFunc); } @@ -195,19 +210,20 @@ public void CopyUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - int key = NumRecs / 2; + int keyNum = NumRecs / 2, valueNum = keyNum * NumRecs * 10; void do_it() { + var key = SpanByte.FromPinnedVariable(ref keyNum); for (int lap = 0; lap < 2; ++lap) { functions.cancelLocation = CancelLocation.NeedCopyUpdate; - var status = bContext.RMW(key, key * NumRecs * 10); + var status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.IsCanceled); ClassicAssert.AreEqual(CancelLocation.NeedCopyUpdate, functions.lastFunc); functions.cancelLocation = CancelLocation.CopyUpdater; - status = bContext.RMW(key, key * NumRecs * 10); + status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.IsCanceled); ClassicAssert.AreEqual(CancelLocation.CopyUpdater, functions.lastFunc); } @@ -228,11 +244,12 @@ public void InPlaceUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - int key = NumRecs / 2; + int keyNum = NumRecs / 2, valueNum = keyNum * NumRecs * 10; + var key = SpanByte.FromPinnedVariable(ref keyNum); // Note: ExpirationTests tests the combination of CancelOperation and DeleteRecord functions.cancelLocation = CancelLocation.InPlaceUpdater; - var status = bContext.RMW(key, key * NumRecs * 10); + var status = bContext.RMW(key, ref valueNum); ClassicAssert.IsTrue(status.IsCanceled); ClassicAssert.AreEqual(CancelLocation.InPlaceUpdater, functions.lastFunc); } @@ -240,31 +257,35 @@ public void InPlaceUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SingleWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) + public void InitialWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - int key = NumRecs + 1; + int keyNum = NumRecs + 1, valueNum = keyNum * NumRecs * 10; + var key = SpanByte.FromPinnedVariable(ref keyNum); + var value = SpanByte.FromPinnedVariable(ref valueNum); - functions.cancelLocation = CancelLocation.SingleWriter; - var status = bContext.Upsert(key, key * NumRecs * 10); + functions.cancelLocation = CancelLocation.InitialWriter; + var status = bContext.Upsert(key, value); ClassicAssert.IsTrue(status.IsCanceled); - ClassicAssert.AreEqual(CancelLocation.SingleWriter, functions.lastFunc); + ClassicAssert.AreEqual(CancelLocation.InitialWriter, functions.lastFunc); } [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void ConcurrentWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) + public void InPlaceWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - int key = NumRecs / 2; + int keyNum = NumRecs / 2, valueNum = keyNum * NumRecs * 10; + var key = SpanByte.FromPinnedVariable(ref keyNum); + var value = SpanByte.FromPinnedVariable(ref valueNum); - functions.cancelLocation = CancelLocation.ConcurrentWriter; - var status = bContext.Upsert(key, key * NumRecs * 10); + functions.cancelLocation = CancelLocation.InPlaceWriter; + var status = bContext.Upsert(key, value); ClassicAssert.IsTrue(status.IsCanceled); - ClassicAssert.AreEqual(CancelLocation.ConcurrentWriter, functions.lastFunc); + ClassicAssert.AreEqual(CancelLocation.InPlaceWriter, functions.lastFunc); } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/CheckpointManagerTests.cs b/libs/storage/Tsavorite/cs/test/CheckpointManagerTests.cs index 6c21404529e..747ee81cb4a 100644 --- a/libs/storage/Tsavorite/cs/test/CheckpointManagerTests.cs +++ b/libs/storage/Tsavorite/cs/test/CheckpointManagerTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -57,7 +59,7 @@ public async Task CheckpointManagerPurgeCheck([Values] DeviceMode deviceMode) }, StoreFunctions.Create(LongKeyComparer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s = store.NewSession>(new SimpleSimpleFunctions()); + using var s = store.NewSession>(new SimpleLongSimpleFunctions()); var bContext = s.BasicContext; var logCheckpoints = new Dictionary(); @@ -148,4 +150,5 @@ public async Task CheckpointManagerPurgeCheck([Values] DeviceMode deviceMode) TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); } } -} \ No newline at end of file +} +#endif // LOGRECORD_TODO \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/CompletePendingTests.cs b/libs/storage/Tsavorite/cs/test/CompletePendingTests.cs index 64c92320472..c58d4ddf4cc 100644 --- a/libs/storage/Tsavorite/cs/test/CompletePendingTests.cs +++ b/libs/storage/Tsavorite/cs/test/CompletePendingTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.Collections.Generic; using System.IO; using System.Threading.Tasks; @@ -232,7 +234,7 @@ public async ValueTask ReadAndCompleteWithPendingOutput([Values] bool useRMW) foreach (var (key, address) in rmwCopyUpdatedAddresses) { - // ConcurrentReader does not verify the input struct. + // Reader does not verify the input struct. InputStruct inputStruct = default; OutputStruct outputStruct = default; ReadOptions readOptions = default; @@ -254,15 +256,12 @@ public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct i } // Read functions - public override bool SingleReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) + public override bool Reader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) { ClassicAssert.IsFalse(readInfo.RecordInfo.IsNull()); dst.value = value; return true; } - - public override bool ConcurrentReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - => SingleReader(ref key, ref input, ref value, ref dst, ref readInfo); } [Test] @@ -359,4 +358,6 @@ public void ReadPendingWithNoNewKey() ClassicAssert.AreEqual(firstValue * valueMult, outputStruct.value.vfield2, "Should have returned first value"); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ComponentRecoveryTests.cs b/libs/storage/Tsavorite/cs/test/ComponentRecoveryTests.cs index 9083c0feffc..441194034d3 100644 --- a/libs/storage/Tsavorite/cs/test/ComponentRecoveryTests.cs +++ b/libs/storage/Tsavorite/cs/test/ComponentRecoveryTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Threading.Tasks; @@ -189,4 +191,6 @@ public async Task FuzzyIndexRecoveryAsyncTest() Finish_FuzzyIndexRecoveryTest(seed, numAdds, ht_device, ofb_device, hash_table1, hash_table2); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/DeviceLogTests.cs b/libs/storage/Tsavorite/cs/test/DeviceLogTests.cs index 741e0aadd28..f93509671ad 100644 --- a/libs/storage/Tsavorite/cs/test/DeviceLogTests.cs +++ b/libs/storage/Tsavorite/cs/test/DeviceLogTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.Buffers; using System.IO; using System.Linq; @@ -94,7 +96,7 @@ private async ValueTask TsavoriteLogTest1(LogChecksumType logChecksum, IDevice d for (int i = 0; i < numEntries; i++) { - log.Enqueue(entry); + _ = log.Enqueue(entry); } log.CompleteLog(true); @@ -116,7 +118,7 @@ private async ValueTask TsavoriteLogTest1(LogChecksumType logChecksum, IDevice d } break; case TsavoriteLogTestBase.IteratorType.AsyncMemoryOwner: - await foreach ((IMemoryOwner result, int _, long _, long nextAddress) in iter.GetAsyncEnumerable(MemoryPool.Shared)) + await foreach ((IMemoryOwner result, _, _, long nextAddress) in iter.GetAsyncEnumerable(MemoryPool.Shared)) { ClassicAssert.IsTrue(result.Memory.Span.ToArray().Take(entry.Length).SequenceEqual(entry)); result.Dispose(); @@ -140,4 +142,6 @@ private async ValueTask TsavoriteLogTest1(LogChecksumType logChecksum, IDevice d log.Dispose(); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ExpirationTests.cs b/libs/storage/Tsavorite/cs/test/ExpirationTests.cs index 8bfa751e9f0..9eef4fc5ce8 100644 --- a/libs/storage/Tsavorite/cs/test/ExpirationTests.cs +++ b/libs/storage/Tsavorite/cs/test/ExpirationTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using NUnit.Framework; @@ -10,7 +12,7 @@ namespace Tsavorite.test.Expiration { - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; [TestFixture] internal class ExpirationTests @@ -33,8 +35,7 @@ internal class ExpirationTests [Flags] internal enum Funcs { - Invalid = 0, NeedInitialUpdate = 0x0001, NeedCopyUpdate = 0x0002, InPlaceUpdater = 0x0004, InitialUpdater = 0x0008, CopyUpdater = 0x0010, - SingleReader = 0x0020, ConcurrentReader = 0x0040, + Invalid = 0, NeedInitialUpdate = 0x0001, NeedCopyUpdate = 0x0002, InPlaceUpdater = 0x0004, InitialUpdater = 0x0008, CopyUpdater = 0x0010, Reader = 0x0020, RMWCompletionCallback = 0x0100, ReadCompletionCallback = 0x0200, SkippedCopyUpdate = NeedCopyUpdate | RMWCompletionCallback, DidCopyUpdate = NeedCopyUpdate | CopyUpdater, @@ -150,12 +151,12 @@ internal enum TestOp Revivify // TODO - NYI: An Update or RMW operation encounters a tombstoned record of >= size of the new value, so the record is updated. // Test with newsize < space, then again with newsize == original space - // Verify tombstone is revivified on later insert (SingleWriter called within Tsavorite-acquired RecordInfo.SpinLock) + // Verify tombstone is revivified on later insert (InitialWriter called within Tsavorite-acquired RecordInfo.SpinLock) // Verify tombstone is revivified on later simple RMW (IU called within Tsavorite-acquired RecordInfo.SpinLock) #pragma warning restore format }; - public class ExpirationFunctions : SessionFunctionsBase + public class ExpirationFunctions : SessionFunctionsBase { private static unsafe void VerifyValue(int key, ref SpanByte valueSpanByte) { @@ -479,27 +480,12 @@ public override void ReadCompletionCallback(ref SpanByte key, ref ExpirationInpu public override int GetUpsertValueLength(ref SpanByte value, ref ExpirationInput input) => value.TotalSize; // Read functions - public override unsafe bool SingleReader(ref SpanByte key, ref ExpirationInput input, ref SpanByte value, ref ExpirationOutput output, ref ReadInfo readInfo) + public override unsafe bool Reader(ref SpanByte key, ref ExpirationInput input, ref SpanByte value, ref ExpirationOutput output, ref ReadInfo readInfo) { int key1 = key.AsSpan()[0]; ref int field1 = ref value.AsSpan()[0]; - output.AddFunc(Funcs.SingleReader); - if (IsExpired(key1, field1)) - { - readInfo.Action = ReadAction.Expire; - return false; - } - output.retrievedValue = field1; - return true; - } - - public override unsafe bool ConcurrentReader(ref SpanByte key, ref ExpirationInput input, ref SpanByte value, ref ExpirationOutput output, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - int key1 = key.AsSpan()[0]; - ref int field1 = ref value.AsSpan()[0]; - - output.AddFunc(Funcs.ConcurrentReader); + output.AddFunc(Funcs.Reader); if (IsExpired(key1, field1)) { readInfo.Action = ReadAction.Expire; @@ -510,16 +496,16 @@ public override unsafe bool ConcurrentReader(ref SpanByte key, ref ExpirationInp } // Upsert functions - public override bool SingleWriter(ref SpanByte key, ref ExpirationInput input, ref SpanByte src, ref SpanByte dst, ref ExpirationOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref SpanByte key, ref ExpirationInput input, ref SpanByte src, ref SpanByte dst, ref ExpirationOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => SpanByteFunctions.DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo); - public override bool ConcurrentWriter(ref SpanByte key, ref ExpirationInput input, ref SpanByte src, ref SpanByte dst, ref ExpirationOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref SpanByte key, ref ExpirationInput input, ref SpanByte src, ref SpanByte dst, ref ExpirationOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => SpanByteFunctions.DoSafeCopy(ref src, ref dst, ref upsertInfo, ref recordInfo); } IDevice log; ExpirationFunctions functions; - TsavoriteKV> store; + TsavoriteKV> store; ClientSession> session; BasicContext> bContext; @@ -634,9 +620,9 @@ private void InitialRead(FlushMode flushMode, bool afterIncrement) ClassicAssert.AreEqual(GetValue(ModifyKey) + (afterIncrement ? 1 : 0), output.retrievedValue); Funcs expectedFuncs = flushMode switch { - FlushMode.NoFlush => Funcs.ConcurrentReader, - FlushMode.ReadOnly => Funcs.SingleReader, - FlushMode.OnDisk => Funcs.SingleReader | Funcs.ReadCompletionCallback, + FlushMode.NoFlush => Funcs.Reader, + FlushMode.ReadOnly => Funcs.Reader, + FlushMode.OnDisk => Funcs.Reader | Funcs.ReadCompletionCallback, _ => Funcs.Invalid }; ClassicAssert.AreNotEqual(expectedFuncs, Funcs.Invalid, $"Unexpected flushmode {flushMode}"); @@ -1126,4 +1112,6 @@ public void DeleteIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Pha GetRecord(key, new(StatusCode.NotFound), flushMode); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/FlakyDeviceTests.cs b/libs/storage/Tsavorite/cs/test/FlakyDeviceTests.cs index afc8b00ff62..c6f1450e9ab 100644 --- a/libs/storage/Tsavorite/cs/test/FlakyDeviceTests.cs +++ b/libs/storage/Tsavorite/cs/test/FlakyDeviceTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Buffers; using System.Collections.Generic; @@ -206,4 +208,6 @@ public async ValueTask FlakyLogTestTolerateFailure([Values] IteratorType iterato } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/FunctionPerSessionTests.cs b/libs/storage/Tsavorite/cs/test/FunctionPerSessionTests.cs index 25e90c780c5..c35cc0f8f66 100644 --- a/libs/storage/Tsavorite/cs/test/FunctionPerSessionTests.cs +++ b/libs/storage/Tsavorite/cs/test/FunctionPerSessionTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using System.Threading; using NUnit.Framework; @@ -95,13 +97,7 @@ public override bool CopyUpdater(ref int key, ref Empty input, ref RefCountedVal public class RefCountedReader : SessionFunctionsBase { - public override bool SingleReader(ref int key, ref Empty input, ref RefCountedValueStruct value, ref RefCountedValueStruct dst, ref ReadInfo readInfo) - { - dst = value; - return true; - } - - public override bool ConcurrentReader(ref int key, ref Empty input, ref RefCountedValueStruct value, ref RefCountedValueStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref int key, ref Empty input, ref RefCountedValueStruct value, ref RefCountedValueStruct dst, ref ReadInfo readInfo) { dst = value; return true; @@ -185,4 +181,6 @@ public void Should_create_multiple_sessions_with_different_callbacks() ClassicAssert.AreEqual(1, _remover.CopyCount); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericByteArrayTests.cs b/libs/storage/Tsavorite/cs/test/GenericByteArrayTests.cs index 14c9cfe9ce7..5f97c0962d1 100644 --- a/libs/storage/Tsavorite/cs/test/GenericByteArrayTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericByteArrayTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Linq; @@ -92,7 +94,7 @@ public void ByteArrayBasicTest() } } - class MyByteArrayFuncs : SimpleSimpleFunctions + class MyByteArrayFuncs : SimpleLongSimpleFunctions { public override void ReadCompletionCallback(ref byte[] key, ref byte[] input, ref byte[] output, Empty ctx, Status status, RecordMetadata recordMetadata) { @@ -116,4 +118,6 @@ public unsafe long GetHashCode64(ref byte[] k) } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericDiskDeleteTests.cs b/libs/storage/Tsavorite/cs/test/GenericDiskDeleteTests.cs index 54c995e78a5..043122fcc0b 100644 --- a/libs/storage/Tsavorite/cs/test/GenericDiskDeleteTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericDiskDeleteTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -184,4 +186,6 @@ public void DiskDeleteBasicTest2() ClassicAssert.AreEqual(input.value, output.value.value); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericIterationTests.cs b/libs/storage/Tsavorite/cs/test/GenericIterationTests.cs index 7b3b5610074..a5c8c391a0a 100644 --- a/libs/storage/Tsavorite/cs/test/GenericIterationTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericIterationTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -72,7 +74,7 @@ internal struct GenericPushIterationTestFunctions : IScanIteratorFunctions 0) @@ -80,8 +82,6 @@ public bool SingleReader(ref MyKey key, ref MyValue value, RecordMetadata record return stopAt != ++numRecords; } - public bool ConcurrentReader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); public readonly bool OnStart(long beginAddress, long endAddress) => true; public readonly void OnException(Exception exception, long numberOfRecords) { } public readonly void OnStop(bool completed, long numberOfRecords) { } @@ -107,7 +107,7 @@ void iterateAndVerify(int keyMultToValue, int expectedRecs) { using var iter = session.Iterate(); while (iter.GetNext(out var recordInfo)) - _ = scanIteratorFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); + _ = scanIteratorFunctions.Reader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); } else ClassicAssert.IsTrue(session.Iterate(ref scanIteratorFunctions), $"Failed to complete push iteration; numRecords = {scanIteratorFunctions.numRecords}"); @@ -257,4 +257,6 @@ void LocalUpdate(int tid) Task.WaitAll([.. tasks]); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericLogCompactionTests.cs b/libs/storage/Tsavorite/cs/test/GenericLogCompactionTests.cs index 6c826264849..2e0186733e9 100644 --- a/libs/storage/Tsavorite/cs/test/GenericLogCompactionTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericLogCompactionTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -26,7 +28,7 @@ public void Setup() // Clean up log files from previous test runs in case they weren't cleaned up DeleteDirectory(MethodTestDir, wait: true); - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = 1L << 13, MutableFraction = 0.1, @@ -343,4 +345,6 @@ private struct EvenCompactionFunctions : ICompactionFunctions public readonly bool IsDeleted(ref MyKey key, ref MyValue value) => value.value % 2 != 0; } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericLogScanTests.cs b/libs/storage/Tsavorite/cs/test/GenericLogScanTests.cs index 2c86d8a99ea..0f9025c0f1f 100644 --- a/libs/storage/Tsavorite/cs/test/GenericLogScanTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericLogScanTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using NUnit.Framework; @@ -79,10 +81,7 @@ internal struct GenericPushScanTestFunctions : IScanIteratorFunctions true; - public bool ConcurrentReader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - - public bool SingleReader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) { cursorRecordResult = CursorRecordResult.Accept; // default; not used here ClassicAssert.AreEqual(numRecords, key.key, $"log scan 1: key"); @@ -143,7 +142,7 @@ void scanAndVerify(ScanBufferingMode sbm) { using var iter = store.Log.Scan(start, store.Log.TailAddress, sbm); while (iter.GetNext(out var recordInfo)) - _ = scanIteratorFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); + _ = scanIteratorFunctions.Reader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); } else ClassicAssert.IsTrue(store.Log.Scan(ref scanIteratorFunctions, start, store.Log.TailAddress, sbm), "Failed to complete push iteration"); @@ -246,10 +245,10 @@ public class ScanFunctions : MyFunctions // Right now this is unused but helped with debugging so I'm keeping it around. internal long insertedAddress; - public override bool SingleWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { insertedAddress = upsertInfo.Address; - return base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + return base.InitialWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); } } @@ -423,7 +422,7 @@ internal void Initialize(bool verifyKeys, Func filter) this.filter = filter; } - public bool ConcurrentReader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) { cursorRecordResult = filter(key) ? CursorRecordResult.Accept : CursorRecordResult.Skip; if (cursorRecordResult != CursorRecordResult.Accept) @@ -443,10 +442,8 @@ public void OnException(Exception exception, long numberOfRecords) public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } - - public bool SingleReader(ref MyKey key, ref MyValue value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => ConcurrentReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); } - } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/GenericStringTests.cs b/libs/storage/Tsavorite/cs/test/GenericStringTests.cs index 4b416f7ae1b..4573e34564f 100644 --- a/libs/storage/Tsavorite/cs/test/GenericStringTests.cs +++ b/libs/storage/Tsavorite/cs/test/GenericStringTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -96,7 +98,7 @@ public void StringBasicTest([Values] DeviceType deviceType) } } - class MyFuncs : SimpleSimpleFunctions + class MyFuncs : SimpleLongSimpleFunctions { public override void ReadCompletionCallback(ref string key, ref string input, ref string output, Empty ctx, Status status, RecordMetadata recordMetadata) { @@ -105,4 +107,6 @@ public override void ReadCompletionCallback(ref string key, ref string input, re } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/InputOutputParameterTests.cs b/libs/storage/Tsavorite/cs/test/InputOutputParameterTests.cs index bb98c372559..2bf59f97f86 100644 --- a/libs/storage/Tsavorite/cs/test/InputOutputParameterTests.cs +++ b/libs/storage/Tsavorite/cs/test/InputOutputParameterTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -27,14 +29,8 @@ internal class UpsertInputFunctions : SessionFunctionsBase - public override bool SingleReader(ref int key, ref int input, ref int value, ref int output, ref ReadInfo readInfo) + public override bool Reader(ref int key, ref int input, ref int value, ref int output, ref ReadInfo readInfo) { ClassicAssert.AreEqual(key * input, value); lastWriteAddress = readInfo.Address; @@ -43,18 +39,18 @@ public override bool SingleReader(ref int key, ref int input, ref int value, ref } /// - public override bool ConcurrentWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - => SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, WriteReason.Upsert, ref recordInfo); + public override bool InPlaceWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + => InitialWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo); /// - public override bool SingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { lastWriteAddress = upsertInfo.Address; dst = output = src * input; return true; } /// - public override void PostSingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reasons) + public override void PostInitialWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo) { ClassicAssert.AreEqual(lastWriteAddress, upsertInfo.Address); ClassicAssert.AreEqual(key * input, dst); @@ -162,9 +158,11 @@ void doReads() loading = false; input *= input; - // ConcurrentWriter (update existing records) + // InPlaceWriter (update existing records) doWrites(); doReads(); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/InsertAtTailSpanByteStressTests.cs b/libs/storage/Tsavorite/cs/test/InsertAtTailSpanByteStressTests.cs index 747b9a23532..eeb510169ea 100644 --- a/libs/storage/Tsavorite/cs/test/InsertAtTailSpanByteStressTests.cs +++ b/libs/storage/Tsavorite/cs/test/InsertAtTailSpanByteStressTests.cs @@ -15,7 +15,7 @@ namespace Tsavorite.test.InsertAtTailStressTests { - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; // Number of mutable pages for this test public enum MutablePages @@ -27,9 +27,9 @@ public enum MutablePages class SpanByteInsertAtTailChainTests { - private TsavoriteKV> store; + private TsavoriteKV> store; private IDevice log; - SpanByteComparerModulo comparer; + SpanByteKeyComparerModulo comparer; const long ValueAdd = 1_000_000_000; const long NumKeys = 2_000; @@ -67,8 +67,8 @@ public void Setup() } // Make the main log mutable region small enough that we force the readonly region to stay close to tail, causing inserts. - int pageBits = 15, memoryBits = 34; - KVSettings kvSettings = new() + int pageBits = 15, memoryBits = 24; + KVSettings kvSettings = new() { LogDevice = log, PageSize = 1L << pageBits, @@ -76,11 +76,11 @@ public void Setup() MutableFraction = 8.0 / (1 << (memoryBits - pageBits)), }; store = new(kvSettings - , StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) + , StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - comparer = new SpanByteComparerModulo(modRange); + comparer = new SpanByteKeyComparerModulo(modRange); } [TearDown] @@ -96,40 +96,44 @@ public void TearDown() internal class RmwSpanByteFunctions : SpanByteFunctions { /// - public override bool ConcurrentWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - src.CopyTo(ref dst); - src.CopyTo(ref output, memoryPool); + if (!logRecord.TrySetValueSpan(srcValue, ref sizeInfo)) + return false; + srcValue.CopyTo(ref output, memoryPool); return true; } /// - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - src.CopyTo(ref dst); - src.CopyTo(ref output, memoryPool); + if (!logRecord.TrySetValueSpan(srcValue, ref sizeInfo)) + return false; + srcValue.CopyTo(ref output, memoryPool); return true; } /// - public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - input.CopyTo(ref newValue); + if (!dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo)) + return false; input.CopyTo(ref output, memoryPool); return true; } /// - public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { // The default implementation of IPU simply writes input to destination, if there is space - base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + if (!logRecord.TrySetValueSpan(input.ReadOnlySpan, ref sizeInfo)) + return false; input.CopyTo(ref output, memoryPool); return true; } /// - public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { Assert.Fail("For these tests, InitialUpdater should never be called"); return false; @@ -138,16 +142,15 @@ public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref Sp unsafe void PopulateAndSetReadOnlyToTail() { - using var session = store.NewSession>(new SpanByteFunctions()); + using var session = store.NewSession>(new SpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[sizeof(long)]; for (long ii = 0; ii < NumKeys; ii++) { - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - var status = bContext.Upsert(ref key, ref key); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + var status = bContext.Upsert(key, key); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } bContext.CompletePending(true); @@ -174,11 +177,10 @@ public void SpanByteTailInsertMultiThreadTest([Values] HashModulo modRange, [Val const int numIterations = 10; unsafe void runReadThread(int tid) { - using var session = store.NewSession>(new SpanByteFunctions()); + using var session = store.NewSession>(new SpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[sizeof(long)]; for (var iteration = 0; iteration < numIterations; ++iteration) { @@ -187,8 +189,8 @@ unsafe void runReadThread(int tid) { SpanByteAndMemory output = default; - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - var status = bContext.Read(ref key, ref output); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + var status = bContext.Read(key, ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -199,7 +201,7 @@ unsafe void runReadThread(int tid) ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {ii}, {status}, wasPending {false}, pt 1"); ClassicAssert.IsNotNull(output.Memory, $"tid {tid}, key {ii}, wasPending {false}, pt 2"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.Span); ClassicAssert.AreEqual(ii, value % ValueAdd, $"tid {tid}, key {ii}, wasPending {false}, pt 3"); output.Memory.Dispose(); } @@ -216,13 +218,13 @@ unsafe void runReadThread(int tid) status = completedOutputs.Current.Status; output = completedOutputs.Current.Output; // Note: do NOT overwrite 'key' here - long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key.AsReadOnlySpan()); + long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key); ClassicAssert.AreEqual(completedOutputs.Current.RecordMetadata.Address == Constants.kInvalidAddress, status.Record.CopiedToReadCache, $"key {keyLong}: {status}"); ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {keyLong}, {status}, wasPending {true}, pt 1"); ClassicAssert.IsNotNull(output.Memory, $"tid {tid}, key {keyLong}, wasPending {true}, pt 2"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.Span); ClassicAssert.AreEqual(keyLong, value % ValueAdd, $"tid {tid}, key {keyLong}, wasPending {true}, pt 3"); output.Memory.Dispose(); } @@ -235,13 +237,12 @@ unsafe void runReadThread(int tid) unsafe void runUpdateThread(int tid) { - using var session = store.NewSession>(new RmwSpanByteFunctions()); + using var session = store.NewSession>(new RmwSpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); - Span inputVec = stackalloc byte[sizeof(long)]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[sizeof(long)]; + Span input = stackalloc byte[sizeof(long)]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); for (var iteration = 0; iteration < numIterations; ++iteration) { @@ -250,11 +251,11 @@ unsafe void runUpdateThread(int tid) { SpanByteAndMemory output = default; - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(inputVec, ii + ValueAdd)); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(input, ii + ValueAdd)); var status = updateOp == UpdateOp.RMW - ? bContext.RMW(ref key, ref input, ref output) - : bContext.Upsert(ref key, ref input, ref input, ref output); + ? bContext.RMW(key, ref pinnedInputSpan, ref output) + : bContext.Upsert(key, ref pinnedInputSpan, input, ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -268,7 +269,7 @@ unsafe void runUpdateThread(int tid) if (updateOp == UpdateOp.RMW) // Upsert will not try to find records below HeadAddress, but it may find them in-memory ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {ii}, {status}"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.Span); ClassicAssert.AreEqual(ii + ValueAdd, value, $"tid {tid}, key {ii}, wasPending {false}"); output.Memory?.Dispose(); @@ -286,12 +287,12 @@ unsafe void runUpdateThread(int tid) status = completedOutputs.Current.Status; output = completedOutputs.Current.Output; // Note: do NOT overwrite 'key' here - long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key.AsReadOnlySpan()); + long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key); if (updateOp == UpdateOp.RMW) // Upsert will not try to find records below HeadAddress, but it may find them in-memory ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {keyLong}, {status}"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.Span); ClassicAssert.AreEqual(keyLong + ValueAdd, value, $"tid {tid}, key {keyLong}, wasPending {true}"); output.Memory?.Dispose(); diff --git a/libs/storage/Tsavorite/cs/test/LargeObjectTests.cs b/libs/storage/Tsavorite/cs/test/LargeObjectTests.cs index 952b5826410..b62850fbbea 100644 --- a/libs/storage/Tsavorite/cs/test/LargeObjectTests.cs +++ b/libs/storage/Tsavorite/cs/test/LargeObjectTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Threading.Tasks; @@ -104,4 +106,6 @@ public async ValueTask LargeObjectTest( } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/LogFastCommitTests.cs b/libs/storage/Tsavorite/cs/test/LogFastCommitTests.cs index 816600ec3e6..0dd9526dfb1 100644 --- a/libs/storage/Tsavorite/cs/test/LogFastCommitTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogFastCommitTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -153,4 +155,6 @@ public void CommitRecordBoundedGrowthTest([Values] TestUtils.DeviceType deviceTy log.Commit(true); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/LogRecordTests.cs b/libs/storage/Tsavorite/cs/test/LogRecordTests.cs new file mode 100644 index 00000000000..7ac01236472 --- /dev/null +++ b/libs/storage/Tsavorite/cs/test/LogRecordTests.cs @@ -0,0 +1,464 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.InteropServices; +using NUnit.Framework; +using Tsavorite.core; +using static Tsavorite.core.Utility; +using static Tsavorite.test.TestUtils; + +namespace Tsavorite.test +{ + /// + /// This also tests and , + /// which in turn tests . + /// + [TestFixture] + unsafe class LogRecordTests + { + long nativePointer; + ObjectIdMap objectIdMap; + SectorAlignedBufferPool bufferPool; + SectorAlignedMemory recordBuffer; + + [SetUp] + public void Setup() + { + DeleteDirectory(MethodTestDir); + objectIdMap = new(); + } + + [TearDown] + public void TearDown() + { + objectIdMap.Clear(); + if (nativePointer != IntPtr.Zero) + { + NativeMemory.AlignedFree((void*)nativePointer); + nativePointer = IntPtr.Zero; + } + recordBuffer?.Return(); + recordBuffer = null; + bufferPool = null; + 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) + { + if (keySize > 0) + sizeInfo.FieldInfo.KeyDataSize = keySize; + if (valueSize > 0) + sizeInfo.FieldInfo.ValueDataSize = valueSize; + + // Key + sizeInfo.KeyIsInline = sizeInfo.FieldInfo.KeyDataSize <= maxInlineKeySize; + keySize = sizeInfo.KeyIsInline ? sizeInfo.FieldInfo.KeyDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + + // Value + sizeInfo.MaxInlineValueSpanSize = maxInlineValueSize; + sizeInfo.ValueIsInline = !sizeInfo.ValueIsObject && sizeInfo.FieldInfo.ValueDataSize <= maxInlineValueSize; + valueSize = sizeInfo.ValueIsInline ? sizeInfo.FieldInfo.ValueDataSize + LogField.InlineLengthPrefixSize : ObjectIdMap.ObjectIdSize; + + // Record + sizeInfo.ActualInlineRecordSize = RecordInfo.GetLength() + keySize + valueSize + sizeInfo.OptionalSize; + sizeInfo.AllocatedInlineRecordSize = RoundUp(sizeInfo.ActualInlineRecordSize, Constants.kRecordAlignment); + } + + [Test] + [Category(LogRecordCategory), Category(SmokeTestCategory)] + //[Repeat(900)] + public unsafe void InlineBasicTest() + { + Span key = stackalloc byte[initialKeyLen]; + Span value = stackalloc byte[initialValueLen]; + + key.Fill(0x42); + value.Fill(0x43); + + var sizeInfo = new RecordSizeInfo(); + InitializeRecord(key, value, ref sizeInfo, out var logRecord, out var expectedFillerLengthAddress, out var expectedFillerLength, out long eTag, out long expiration); + + // Shrink + var offset = 12; + sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + Assert.That(logRecord.TrySetValueLength(ref sizeInfo), Is.True); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + offset)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + + // Grow within range + offset = 6; + sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + Assert.That(logRecord.TrySetValueLength(ref sizeInfo), Is.True); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + offset)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + + // Grow beyond range + offset = -10; + sizeInfo.FieldInfo.ValueDataSize = initialValueLen - offset; + Assert.That(logRecord.TrySetValueLength(ref sizeInfo), Is.False); + + // Restore to original + sizeInfo.FieldInfo.ValueDataSize = initialValueLen; + Assert.That(logRecord.TrySetValueLength(ref sizeInfo), Is.True); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + + // Remove ETag and verify Expiration is the same and filler has grown. + Assert.That(logRecord.RemoveETag(), Is.True); + Assert.That(logRecord.Info.HasETag, Is.False); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - LogRecord.ETagSize)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + LogRecord.ETagSize)); + + // Restore ETag and verify Expiration is the same and filler has grown. + eTag += 10; + Assert.That(logRecord.TrySetETag(eTag), Is.True); + Assert.That(logRecord.Info.HasETag, Is.True); + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + // Remove Expiration and verify ETag is the same and filler has grown. + Assert.That(logRecord.RemoveExpiration(), Is.True); + Assert.That(logRecord.Info.HasExpiration, Is.False); + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - LogRecord.ExpirationSize)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + LogRecord.ExpirationSize)); + + // Restore Expiration and verify ETag is the same and filler has grown. + expiration += 20; + Assert.That(logRecord.TrySetExpiration(expiration), Is.True); + Assert.That(logRecord.Info.HasExpiration, Is.True); + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + } + + [Test] + [Category(LogRecordCategory), Category(SmokeTestCategory)] + //[Repeat(900)] + public unsafe void ConversionTest() + { + 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); + + var sizeInfo = new RecordSizeInfo(); + InitializeRecord(key, value, ref sizeInfo, out var logRecord, out var expectedFillerLengthAddress, out var expectedFillerLength, out long eTag, out long expiration); + + // 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); + RestoreToOriginal(value, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, 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); + RestoreToOriginal(value, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration); + + // Convert to overflow, then to object, then back to overflow and back to original + ConvertToOverflow(overflowValue, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration, offset); + ConvertToObject(ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration, offset); + ConvertToOverflow(overflowValue, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration, offset); + RestoreToOriginal(value, ref sizeInfo, ref logRecord, expectedFillerLengthAddress, expectedFillerLength, eTag, expiration); + } + + [Test] + [Category(LogRecordCategory), Category(SmokeTestCategory)] + //[Repeat(900)] + public unsafe void SerializationTest() + { + 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(ref 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(ref 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); + + // 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, ref 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(ref 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(ref 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))); + } + } + + 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, + HasETag = true, + HasExpiration = true + }; + + UpdateRecordSizeInfo(ref sizeInfo); + Assert.That(sizeInfo.ActualInlineRecordSize, Is.EqualTo(expectedInitialActualInlineRecordSize)); + Assert.That(sizeInfo.AllocatedInlineRecordSize, Is.EqualTo(expectedInitialAllocatedInlineRecordSize)); + Assert.That(sizeInfo.KeyIsInline, Is.True); + Assert.That(sizeInfo.ValueIsInline, Is.True); + + nativePointer = (long)NativeMemory.AlignedAlloc((nuint)sizeInfo.AllocatedInlineRecordSize, Constants.kCacheLineBytes); + long recordEndAddress = nativePointer + sizeInfo.AllocatedInlineRecordSize; + + logRecord = new LogRecord(nativePointer, objectIdMap); + logRecord.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); + + // 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; + expectedFillerLength = recordEndAddress - expectedFillerLengthAddress; + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + Assert.That(logRecord.TrySetValueSpan(value, ref sizeInfo), Is.True); + + Assert.That(logRecord.Info.ValueIsInline, Is.True); + Assert.That(logRecord.Info.ValueIsOverflow, Is.False); + Assert.That(logRecord.Info.ValueIsObject, Is.False); + Assert.That(logRecord.ValueSpan.Length, Is.EqualTo(value.Length)); + Assert.That(logRecord.ValueSpan.Slice(0, sizeof(int)).AsRef(), Is.EqualTo(0x43434343)); + + // These should be the same still. + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + eTag = initialETag; + Assert.That(logRecord.TrySetETag(eTag), Is.True); + + expectedFillerLengthAddress += LogRecord.ETagSize; + expectedFillerLength -= LogRecord.ETagSize; + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + expiration = InitialExpiration; + Assert.That(logRecord.TrySetExpiration(expiration), Is.True); + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + + expectedFillerLengthAddress += LogRecord.ExpirationSize; + expectedFillerLength -= LogRecord.ExpirationSize; + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + } + + 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.ValueIsObject = false; + UpdateRecordSizeInfo(ref sizeInfo); + + Assert.That(logRecord.TrySetValueSpan(overflowValue, ref sizeInfo), Is.True); + + Assert.That(logRecord.Info.ValueIsInline, Is.False); + Assert.That(logRecord.Info.ValueIsOverflow, Is.True); + Assert.That(logRecord.ValueSpan.Length, Is.EqualTo(overflowValue.Length)); + Assert.That(logRecord.ValueSpan.Slice(0, sizeof(int)).AsRef(), Is.EqualTo(0x53535353)); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + offset)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + } + + 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.ValueIsObject = true; + UpdateRecordSizeInfo(ref sizeInfo); + + var valueObject = new TestObjectValue() { value = 0x63636363 }; + Assert.That(logRecord.TrySetValueObject(valueObject, ref sizeInfo), Is.True); + + Assert.That(logRecord.Info.ValueIsInline, Is.False); + Assert.That(logRecord.Info.ValueIsOverflow, Is.False); + Assert.That(logRecord.Info.ValueIsObject, Is.True); + Assert.That(((TestObjectValue)logRecord.ValueObject).value, Is.EqualTo(0x63636363)); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress - offset)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength + offset)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + } + + 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.ValueIsObject = false; + UpdateRecordSizeInfo(ref sizeInfo); + + Assert.That(logRecord.TrySetValueSpan(value, ref sizeInfo), Is.True); + + Assert.That(logRecord.Info.ValueIsInline, Is.True); + Assert.That(logRecord.Info.ValueIsOverflow, Is.False); + Assert.That(logRecord.ValueSpan.Length, Is.EqualTo(value.Length)); + Assert.That(logRecord.ValueSpan.Slice(0, sizeof(int)).AsRef(), Is.EqualTo(0x43434343)); + + Assert.That(logRecord.GetFillerLengthAddress(), Is.EqualTo(expectedFillerLengthAddress)); + Assert.That(logRecord.GetFillerLength(), Is.EqualTo(expectedFillerLength)); + + Assert.That(logRecord.ETag, Is.EqualTo(eTag)); + Assert.That(logRecord.Expiration, Is.EqualTo(expiration)); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/LogResumeTests.cs b/libs/storage/Tsavorite/cs/test/LogResumeTests.cs index 5abbae37427..81dafcc10e3 100644 --- a/libs/storage/Tsavorite/cs/test/LogResumeTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogResumeTests.cs @@ -22,7 +22,7 @@ public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - device = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "Tsavoritelog.log"), deleteOnClose: true); + device = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "TsavoriteLog.log"), deleteOnClose: true); } [TearDown] diff --git a/libs/storage/Tsavorite/cs/test/LogScanTests.cs b/libs/storage/Tsavorite/cs/test/LogScanTests.cs index 357c4a19fe6..253eb7bba3b 100644 --- a/libs/storage/Tsavorite/cs/test/LogScanTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogScanTests.cs @@ -238,7 +238,7 @@ public void ScanNoDefaultTest([Values] TestUtils.DeviceType deviceType) // Read the log - Look for the flag so know each entry is unique int currentEntry = 0; - using (var iter = log.Scan(0, 100_000_000, recover: true, scanBufferingMode: ScanBufferingMode.DoublePageBuffering, scanUncommitted: false)) + using (var iter = log.Scan(0, 100_000_000, recover: true, scanBufferingMode: DiskScanBufferingMode.DoublePageBuffering, scanUncommitted: false)) { while (iter.GetNext(out byte[] result, out _, out _)) { @@ -334,7 +334,7 @@ public void ScanBufferingModeDoublePageTest([Values] TestUtils.DeviceType device // Read the log - Look for the flag so know each entry is unique int currentEntry = 0; - using (var iter = log.Scan(0, 100_000_000, scanBufferingMode: ScanBufferingMode.DoublePageBuffering)) + using (var iter = log.Scan(0, 100_000_000, scanBufferingMode: DiskScanBufferingMode.DoublePageBuffering)) { while (iter.GetNext(out byte[] result, out _, out _)) { @@ -364,7 +364,7 @@ public void ScanBufferingModeSinglePageTest([Values] TestUtils.DeviceType device // Read the log - Look for the flag so know each entry is unique int currentEntry = 0; - using (var iter = log.Scan(0, 100_000_000, scanBufferingMode: ScanBufferingMode.SinglePageBuffering)) + using (var iter = log.Scan(0, 100_000_000, scanBufferingMode: DiskScanBufferingMode.SinglePageBuffering)) { while (iter.GetNext(out byte[] result, out _, out _)) { diff --git a/libs/storage/Tsavorite/cs/test/LogShiftTailStressTest.cs b/libs/storage/Tsavorite/cs/test/LogShiftTailStressTest.cs index b30f7501946..c0d292cada7 100644 --- a/libs/storage/Tsavorite/cs/test/LogShiftTailStressTest.cs +++ b/libs/storage/Tsavorite/cs/test/LogShiftTailStressTest.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.Threading; @@ -67,4 +69,6 @@ public void TsavoriteLogShiftTailStressTest() log.Commit(true); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/LogTests.cs b/libs/storage/Tsavorite/cs/test/LogTests.cs index 40c6eea9418..868081bd83c 100644 --- a/libs/storage/Tsavorite/cs/test/LogTests.cs +++ b/libs/storage/Tsavorite/cs/test/LogTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Buffers; using System.Collections.Generic; @@ -1214,4 +1216,6 @@ public async ValueTask TsavoriteLogAsyncConsumerTestAfterDisposeIterator([Values } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/LowMemoryTests.cs b/libs/storage/Tsavorite/cs/test/LowMemoryTests.cs index 3c553f3ebff..a4d0ddb5020 100644 --- a/libs/storage/Tsavorite/cs/test/LowMemoryTests.cs +++ b/libs/storage/Tsavorite/cs/test/LowMemoryTests.cs @@ -5,25 +5,26 @@ using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; +using static Tsavorite.test.TestUtils; namespace Tsavorite.test.LowMemory { - using LongAllocator = BlittableAllocator>>; - using LongStoreFunctions = StoreFunctions>; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; [TestFixture] public class LowMemoryTests { IDevice log; - TsavoriteKV store1; + TsavoriteKV store1; const int NumOps = 2000; [SetUp] public void Setup() { - TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - log = new LocalMemoryDevice(1L << 28, 1L << 25, 1, latencyMs: 20, fileName: Path.Join(TestUtils.MethodTestDir, "test.log")); - _ = Directory.CreateDirectory(TestUtils.MethodTestDir); + DeleteDirectory(MethodTestDir, wait: true); + log = new LocalMemoryDevice(1L << 28, 1L << 25, 1, latencyMs: 20, fileName: Path.Join(MethodTestDir, "test.log")); + _ = Directory.CreateDirectory(MethodTestDir); store1 = new(new() { IndexSize = 1L << 16, @@ -32,8 +33,8 @@ public void Setup() PageSize = 1L << 10, MemorySize = 1L << 12, SegmentSize = 1L << 26, - CheckpointDir = TestUtils.MethodTestDir - }, StoreFunctions.Create(LongKeyComparer.Instance) + CheckpointDir = MethodTestDir + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -45,22 +46,22 @@ public void TearDown() store1 = null; log?.Dispose(); log = null; - TestUtils.DeleteDirectory(TestUtils.MethodTestDir); + DeleteDirectory(MethodTestDir); } - private static void Populate(ClientSession, LongStoreFunctions, LongAllocator> s1) + private static void Populate(ClientSession s1) { var bContext1 = s1.BasicContext; for (long key = 0; key < NumOps; key++) - _ = bContext1.Upsert(ref key, ref key); + _ = bContext1.Upsert(SpanByte.FromPinnedVariable(ref key), SpanByte.FromPinnedVariable(ref key)); } [Test] [Category("TsavoriteKV")] - [Category(TestUtils.StressTestCategory)] + [Category(StressTestCategory)] public void LowMemConcurrentUpsertReadTest() { - using var s1 = store1.NewSession>(new SimpleSimpleFunctions((a, b) => a + b)); + using var s1 = store1.NewSession(new SimpleLongSimpleFunctions((a, b) => a + b)); var bContext1 = s1.BasicContext; Populate(s1); @@ -69,7 +70,7 @@ public void LowMemConcurrentUpsertReadTest() var numCompleted = 0; for (long key = 0; key < NumOps; key++) { - var (status, output) = bContext1.Read(key); + var (status, output) = bContext1.Read(SpanByte.FromPinnedVariable(ref key)); if (!status.IsPending) { ++numCompleted; @@ -85,7 +86,7 @@ public void LowMemConcurrentUpsertReadTest() { ++numCompleted; ClassicAssert.IsTrue(completedOutputs.Current.Status.Found, $"{completedOutputs.Current.Status}"); - ClassicAssert.AreEqual(completedOutputs.Current.Key, completedOutputs.Current.Output); + ClassicAssert.AreEqual(completedOutputs.Current.Key.AsRef(), completedOutputs.Current.Output); } } ClassicAssert.AreEqual(NumOps, numCompleted, "numCompleted"); @@ -93,10 +94,10 @@ public void LowMemConcurrentUpsertReadTest() [Test] [Category("TsavoriteKV")] - [Category(TestUtils.StressTestCategory)] + [Category(StressTestCategory)] public void LowMemConcurrentUpsertRMWReadTest([Values] bool completeSync) { - using var s1 = store1.NewSession>(new SimpleSimpleFunctions((a, b) => a + b)); + using var s1 = store1.NewSession(new SimpleLongSimpleFunctions((a, b) => a + b)); var bContext1 = s1.BasicContext; Populate(s1); @@ -105,7 +106,7 @@ public void LowMemConcurrentUpsertRMWReadTest([Values] bool completeSync) int numPending = 0; for (long key = 0; key < NumOps; key++) { - var status = bContext1.RMW(ref key, ref key); + var status = bContext1.RMW(SpanByte.FromPinnedVariable(ref key), ref key); if (status.IsPending && (++numPending % 256) == 0) { _ = bContext1.CompletePending(wait: true); @@ -119,7 +120,7 @@ public void LowMemConcurrentUpsertRMWReadTest([Values] bool completeSync) var numCompleted = 0; for (long key = 0; key < NumOps; key++) { - var (status, output) = bContext1.Read(key); + var (status, output) = bContext1.Read(SpanByte.FromPinnedVariable(ref key)); if (!status.IsPending) { ++numCompleted; @@ -135,7 +136,7 @@ public void LowMemConcurrentUpsertRMWReadTest([Values] bool completeSync) { ++numCompleted; ClassicAssert.IsTrue(completedOutputs.Current.Status.Found, $"{completedOutputs.Current.Status}"); - ClassicAssert.AreEqual(completedOutputs.Current.Key * 2, completedOutputs.Current.Output); + ClassicAssert.AreEqual(completedOutputs.Current.Key.AsRef() * 2, completedOutputs.Current.Output); } } ClassicAssert.AreEqual(NumOps, numCompleted, "numCompleted"); diff --git a/libs/storage/Tsavorite/cs/test/MiscTests.cs b/libs/storage/Tsavorite/cs/test/MiscTests.cs index e7db813fbeb..97b78e7de48 100644 --- a/libs/storage/Tsavorite/cs/test/MiscTests.cs +++ b/libs/storage/Tsavorite/cs/test/MiscTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using NUnit.Framework; @@ -53,69 +55,6 @@ public void TearDown() DeleteDirectory(MethodTestDir); } - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] - public void MixedTest1() - { - using var session = store.NewSession(new MixedFunctions()); - var bContext = session.BasicContext; - - int key = 8999998; - var input1 = new MyInput { value = 23 }; - MyOutput output = new(); - - _ = bContext.RMW(ref key, ref input1, Empty.Default); - - int key2 = 8999999; - var input2 = new MyInput { value = 24 }; - _ = bContext.RMW(ref key2, ref input2, Empty.Default); - - _ = bContext.Read(ref key, ref input1, ref output, Empty.Default); - ClassicAssert.AreEqual(input1.value, output.value.value); - - _ = bContext.Read(ref key2, ref input2, ref output, Empty.Default); - ClassicAssert.AreEqual(input2.value, output.value.value); - } - - [Test] - [Category("TsavoriteKV")] - public void MixedTest2() - { - using var session = store.NewSession(new MixedFunctions()); - var bContext = session.BasicContext; - - for (int i = 0; i < 2000; i++) - { - var value = new MyValue { value = i }; - _ = bContext.Upsert(ref i, ref value, Empty.Default); - } - - var key2 = 23; - MyInput input = new(); - MyOutput g1 = new(); - var status = bContext.Read(ref key2, ref input, ref g1, Empty.Default); - - if (status.IsPending) - { - _ = bContext.CompletePendingWithOutputs(out var outputs, wait: true); - (status, _) = GetSinglePendingResult(outputs); - } - ClassicAssert.IsTrue(status.Found); - - ClassicAssert.AreEqual(23, g1.value.value); - - key2 = 99999; - status = bContext.Read(ref key2, ref input, ref g1, Empty.Default); - - if (status.IsPending) - { - _ = bContext.CompletePendingWithOutputs(out var outputs, wait: true); - (status, _) = GetSinglePendingResult(outputs); - } - ClassicAssert.IsFalse(status.Found); - } - [Test] [Category("TsavoriteKV")] public void ForceRCUAndRecover([Values(UpdateOp.Upsert, UpdateOp.Delete)] UpdateOp updateOp) @@ -155,13 +94,13 @@ public void ForceRCUAndRecover([Values(UpdateOp.Upsert, UpdateOp.Delete)] Update var status = bContext.Upsert(ref key, ref input, ref value, ref output, out RecordMetadata recordMetadata1); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); - // ConcurrentWriter and InPlaceUpater return false, so we create a new record. + // InPlaceWriter and InPlaceUpater return false, so we create a new record. RecordMetadata recordMetadata2; value = new ValueStruct() { vfield1 = 1001, vfield2 = 2002 }; if (updateOp == UpdateOp.Upsert) { status = bContext.Upsert(ref key, ref input, ref value, ref output, out recordMetadata2); - ClassicAssert.AreEqual(1, copyOnWrite.ConcurrentWriterCallCount); + ClassicAssert.AreEqual(1, copyOnWrite.InPlaceWriterCallCount); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } else @@ -215,4 +154,6 @@ public void ForceRCUAndRecover([Values(UpdateOp.Upsert, UpdateOp.Delete)] Update } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ModifiedBitTests.cs b/libs/storage/Tsavorite/cs/test/ModifiedBitTests.cs index ab636dccb93..62f20ce8156 100644 --- a/libs/storage/Tsavorite/cs/test/ModifiedBitTests.cs +++ b/libs/storage/Tsavorite/cs/test/ModifiedBitTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using NUnit.Framework; @@ -34,8 +36,8 @@ class ModifiedBitTests ModifiedBitTestComparer comparer; private TsavoriteKV store; - private ClientSession, IntStoreFunctions, IntAllocator> session; - private BasicContext, IntStoreFunctions, IntAllocator> bContext; + private ClientSession, IntStoreFunctions, IntAllocator> session; + private BasicContext, IntStoreFunctions, IntAllocator> bContext; private IDevice log; [SetUp] @@ -52,7 +54,7 @@ public void Setup() }, StoreFunctions.Create(comparer) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - session = store.NewSession>(new SimpleSimpleFunctions()); + session = store.NewSession>(new SimpleLongSimpleFunctions()); bContext = session.BasicContext; } @@ -73,23 +75,23 @@ void Populate() ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); } - void AssertLockandModified(LockableUnsafeContext, IntStoreFunctions, IntAllocator> luContext, int key, bool xlock, bool slock, bool modified = false) + void AssertLockandModified(TransactionalUnsafeContext, IntStoreFunctions, IntAllocator> luContext, int key, bool xlock, bool slock, bool modified = false) { OverflowBucketLockTableTests.AssertLockCounts(store, ref key, xlock, slock); var isM = luContext.IsModified(key); ClassicAssert.AreEqual(modified, isM, "modified mismatch"); } - void AssertLockandModified(LockableContext, IntStoreFunctions, IntAllocator> luContext, int key, bool xlock, bool slock, bool modified = false) + void AssertLockandModified(TransactionalContext, IntStoreFunctions, IntAllocator> luContext, int key, bool xlock, bool slock, bool modified = false) { OverflowBucketLockTableTests.AssertLockCounts(store, ref key, xlock, slock); var isM = luContext.IsModified(key); ClassicAssert.AreEqual(modified, isM, "modified mismatch"); } - void AssertLockandModified(ClientSession, IntStoreFunctions, IntAllocator> session, int key, bool xlock, bool slock, bool modified = false) + void AssertLockandModified(ClientSession, IntStoreFunctions, IntAllocator> session, int key, bool xlock, bool slock, bool modified = false) { - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); OverflowBucketLockTableTests.AssertLockCounts(store, ref key, xlock, slock); @@ -108,11 +110,11 @@ public void LockAndNotModify() int key = r.Next(NumRecords); bContext.ResetModified(key); - var lContext = session.LockableContext; - lContext.BeginLockable(); + var lContext = session.TransactionalContext; + lContext.BeginTransaction(); AssertLockandModified(lContext, key, xlock: false, slock: false, modified: false); - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Exclusive, lContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key, LockType.Exclusive, lContext) }; lContext.Lock(keyVec); AssertLockandModified(lContext, key, xlock: true, slock: false, modified: false); @@ -127,7 +129,7 @@ public void LockAndNotModify() lContext.Unlock(keyVec); AssertLockandModified(lContext, key, xlock: false, slock: false, modified: false); - lContext.EndLockable(); + lContext.EndTransaction(); } [Test] @@ -200,11 +202,11 @@ public void ModifyLUC([Values(true, false)] bool flushToDisk, [Values] UpdateOp int key = NumRecords - 500; int value = 14; bContext.ResetModified(key); - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); AssertLockandModified(luContext, key, xlock: false, slock: false, modified: false); - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); if (flushToDisk) @@ -213,9 +215,9 @@ public void ModifyLUC([Values(true, false)] bool flushToDisk, [Values] UpdateOp Status status = default; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Exclusive, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key, LockType.Exclusive, luContext) }; luContext.Lock(keyVec); @@ -261,7 +263,7 @@ public void ModifyLUC([Values(true, false)] bool flushToDisk, [Values] UpdateOp AssertLockandModified(luContext, key, xlock: false, slock: false, modified: updateOp != UpdateOp.Delete); - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } @@ -326,11 +328,11 @@ public void ModifyLC([Values(true, false)] bool flushToDisk, [Values] UpdateOp u int key = NumRecords - 500; int value = 14; bContext.ResetModified(key); - var lContext = session.LockableContext; - lContext.BeginLockable(); + var lContext = session.TransactionalContext; + lContext.BeginTransaction(); AssertLockandModified(lContext, key, xlock: false, slock: false, modified: false); - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Exclusive, lContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key, LockType.Exclusive, lContext) }; lContext.Lock(keyVec); @@ -380,7 +382,7 @@ public void ModifyLC([Values(true, false)] bool flushToDisk, [Values] UpdateOp u } AssertLockandModified(lContext, key, xlock: false, slock: false, modified: updateOp != UpdateOp.Delete); - lContext.EndLockable(); + lContext.EndTransaction(); } [Test] @@ -390,16 +392,16 @@ public void CopyToTailTest() Populate(); store.Log.FlushAndEvict(wait: true); - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; int input = 0, output = 0, key = 200; ReadOptions readOptions = new() { CopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.MainLog) }; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); AssertLockandModified(luContext, key, xlock: false, slock: false, modified: true); - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Shared, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key, LockType.Shared, luContext) }; luContext.Lock(keyVec); AssertLockandModified(luContext, key, xlock: false, slock: true, modified: true); @@ -423,8 +425,10 @@ public void CopyToTailTest() luContext.Unlock(keyVec); AssertLockandModified(luContext, key, xlock: false, slock: false, modified: true); - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/MoreLogCompactionTests.cs b/libs/storage/Tsavorite/cs/test/MoreLogCompactionTests.cs index f9c97fdbccf..c24cb4e5c98 100644 --- a/libs/storage/Tsavorite/cs/test/MoreLogCompactionTests.cs +++ b/libs/storage/Tsavorite/cs/test/MoreLogCompactionTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -50,7 +52,7 @@ public void TearDown() public void DeleteCompactLookup([Values] CompactionType compactionType) { - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession>(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; const int totalRecords = 2000; @@ -71,7 +73,7 @@ public void DeleteCompactLookup([Values] CompactionType compactionType) ClassicAssert.AreEqual(compactUntil, store.Log.BeginAddress); - using var session2 = store.NewSession>(new SimpleSimpleFunctions()); + using var session2 = store.NewSession>(new SimpleLongSimpleFunctions()); var bContext2 = session2.BasicContext; // Verify records by reading @@ -98,4 +100,6 @@ public void DeleteCompactLookup([Values] CompactionType compactionType) } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/NativeReadCacheTests.cs b/libs/storage/Tsavorite/cs/test/NativeReadCacheTests.cs index 0926c50163f..289b2ded4e9 100644 --- a/libs/storage/Tsavorite/cs/test/NativeReadCacheTests.cs +++ b/libs/storage/Tsavorite/cs/test/NativeReadCacheTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -234,4 +236,6 @@ public void NativeDiskWriteReadCache2() } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/NeedCopyUpdateTests.cs b/libs/storage/Tsavorite/cs/test/NeedCopyUpdateTests.cs index 2eeab310e66..1713c88de81 100644 --- a/libs/storage/Tsavorite/cs/test/NeedCopyUpdateTests.cs +++ b/libs/storage/Tsavorite/cs/test/NeedCopyUpdateTests.cs @@ -1,11 +1,13 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; -using static Tsavorite.test.NeedCopyUpdateTests; +// using static Tsavorite.test.NeedCopyUpdateTests; using static Tsavorite.test.TestUtils; namespace Tsavorite.test @@ -235,8 +237,10 @@ public void CopyUpdateFromHeadReadOnlyPageTest() } } - internal class RMWSinglePageFunctions : SimpleSimpleFunctions + internal class RMWSinglePageFunctions : SimpleLongSimpleFunctions { } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs b/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs new file mode 100644 index 00000000000..7e33f1533bf --- /dev/null +++ b/libs/storage/Tsavorite/cs/test/ObjectIdMapTests.cs @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading.Tasks; +using NUnit.Framework; +using Tsavorite.core; +using static Tsavorite.test.TestUtils; + +namespace Tsavorite.test +{ + /// + /// This also tests and , + /// which in turn tests . + /// + [TestFixture] + class ObjectIdMapTests + { + ObjectIdMap map; + + [SetUp] + public void Setup() + { + DeleteDirectory(MethodTestDir); + map = new(); + } + + [TearDown] + public void TearDown() + { + DeleteDirectory(MethodTestDir); + } + + [Test] + [Category(ObjectIdMapCategory), Category(MultiLevelPageArrayCategory), Category(SmokeTestCategory)] + public unsafe void ObjectIdMapTest1([Values(1, 8)] int numThreads) + { + Assert.That(map.objectArray.IsInitialized, Is.False); + + // Allocate enough to fill past the first MultiLevelPageArray.InitialBookSize chapters. + var chaptersPerThread = MultiLevelPageArray.InitialBookSize / numThreads + 1; + + void runLoadThread(int tid) + { + // Reduce memory stress by reusing the same object because we are not doing operations on it; it's just a null/not-null indicator in the slot. + var valueObject = new TestObjectValue(); + + for (var chapter = 0; chapter < chaptersPerThread; ++chapter) + { + for (int page = 0; page < MultiLevelPageArray.ChapterSize; ++page) + { + // Assert.That() does reflection and allocates a ConstraintResult class instance, so use a bare test to filter for it in inner loops. + var objectId = map.Allocate(); + if (objectId > map.Count) + Assert.Fail("objectId should be <= map.Count"); + map.Set(objectId, valueObject); + } + } + } + + Task[] tasks = new Task[numThreads]; // Task rather than Thread for propagation of exceptions. + for (int t = 0; t < numThreads; t++) + { + var tid = t; + tasks[t] = Task.Factory.StartNew(() => runLoadThread(tid)); + } + Task.WaitAll(tasks); + + var allocatedCount = map.Count; + Assert.That(allocatedCount, Is.EqualTo(chaptersPerThread * MultiLevelPageArray.ChapterSize * numThreads)); + Assert.That(map.objectArray.book.Length, Is.GreaterThan(MultiLevelPageArray.InitialBookSize)); + + // Now test the freelist loading. + void runLoadFreeListThread(int tid) + { + Random rng = new(tid); + + // Free() from a thread-specific chapter to threads aren't freeing the same objectId; in actual use, + // we'd Allocate() which does per-thread ownership instead. + for (var page = 0; page < MultiLevelPageArray.ChapterSize; ++page) + { + // 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"); + } + } + + Array.Clear(tasks); + for (int t = 0; t < numThreads; t++) + { + var tid = t; + tasks[t] = Task.Factory.StartNew(() => runLoadFreeListThread(tid)); + } + Task.WaitAll(tasks); + + Assert.That(map.freeSlots.MaxCount, Is.EqualTo(MultiLevelPageArray.ChapterSize * numThreads), "All freed items should have been added to the the freeList elementArray"); + Assert.That(map.freeSlots.stack.IsNil, Is.False, "All freed items should be in the stack"); + Assert.That(map.freeSlots.freeNodes.IsNil, Is.True, "No freed items should be in the freeList"); + + // Finally, test the freelist allocation. + void runAllocateFromFreeListThread(int tid) + { + Random rng = new(tid); + + // Free() from a thread-specific chapter to threads aren't freeing the same objectId; in actual use, + // we'd Allocate() which does per-thread ownership instead. + for (var page = 0; page < MultiLevelPageArray.ChapterSize; ++page) + { + var objectId = map.Allocate(); + + // The request should have been satisfied from the freeList, not another allocation. + if (objectId >= allocatedCount) + 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"); + } + } + + Array.Clear(tasks); + for (int t = 0; t < numThreads; t++) + { + var tid = t; + tasks[t] = Task.Factory.StartNew(() => runAllocateFromFreeListThread(tid)); + } + Task.WaitAll(tasks); + + Assert.That(map.Count, Is.EqualTo(allocatedCount)); + + Assert.That(map.freeSlots.stack.IsNil, Is.True, "No freed items should be in the stack"); + Assert.That(map.freeSlots.freeNodes.IsNil, Is.False, "All freed items should be in the freeList"); + Assert.That(map.freeSlots.elementArray.Count, Is.EqualTo(MultiLevelPageArray.ChapterSize * numThreads), "No freed items should have been added to the the freeList elementArray"); + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs b/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs new file mode 100644 index 00000000000..40acd6d0988 --- /dev/null +++ b/libs/storage/Tsavorite/cs/test/ObjectInlineTests.cs @@ -0,0 +1,444 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.IO; +using NUnit.Framework; +using Tsavorite.core; +using static Tsavorite.test.TestUtils; + +namespace Tsavorite.test +{ + using ClassAllocator = ObjectAllocator>; + using ClassStoreFunctions = StoreFunctions; + + [TestFixture] + internal class ObjectInlineTests + { + private TsavoriteKV store; + private IDevice log, objlog; + + [SetUp] + 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); + + store = new(new() + { + IndexSize = 1L << 13, + LogDevice = log, + ObjectLogDevice = objlog, + MutableFraction = 0.1, + MemorySize = 1L << 15, + PageSize = 1L << 10 + }, StoreFunctions.Create(new TestObjectKey.Comparer(), () => new TestObjectValue.Serializer(), DefaultRecordDisposer.Instance) + , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) + ); + } + + [TearDown] + public void TearDown() + { + store?.Dispose(); + store = null; + log?.Dispose(); + log = null; + objlog?.Dispose(); + objlog = null; + DeleteDirectory(MethodTestDir); + } + + [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + public void ObjectAsInlineStructUpsertTest() + { + using var session = store.NewSession(new TestInlineObjectFunctions()); + var bContext = session.BasicContext; + + TestObjectKey keyStruct = new() { key = 9999999 }; + Span key = SpanByte.FromPinnedVariable(ref keyStruct); + TestObjectInput input = new() { value = 23 }; + TestObjectOutput output = default; + + // Overflow<->Inline conversions + + // Start with an inline value. + input.wantValueStyle = TestValueStyle.Inline; + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 24; + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 25; + input.wantValueStyle = TestValueStyle.Overflow; + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 26; + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + // Overflow<->Object conversions + + input.value = 30; + input.wantValueStyle = TestValueStyle.Object; // Overflow -> Object + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 31; + input.wantValueStyle = TestValueStyle.Overflow; // Object -> Overflow + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 32; + input.wantValueStyle = TestValueStyle.Object; // Overflow -> Object again + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + // Object<->Inline conversions + + input.value = 40; + input.wantValueStyle = TestValueStyle.Inline; // Object -> Inline + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 41; + input.wantValueStyle = TestValueStyle.Object; // Inline -> Object + _ = bContext.Upsert(key, ref input, desiredValue: (IHeapObject)null, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + } + + [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] + public void ObjectAsInlineStructRMWTest() + { + using var session = store.NewSession(new TestInlineObjectFunctions()); + var bContext = session.BasicContext; + + TestObjectKey keyStruct = new() { key = 9999999 }; + Span key = SpanByte.FromPinnedVariable(ref keyStruct); + TestObjectInput input = new() { value = 23 }; + TestObjectOutput output = default; + + // Overflow<->Inline conversions + + // Start with an inline value. + input.wantValueStyle = TestValueStyle.Inline; + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.None)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + var priorSum = input.value; + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(input.value)); + + input.value = 24; + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + input.value = 25; + input.wantValueStyle = TestValueStyle.Overflow; + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + input.value = 26; + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + // Overflow<->Object conversions + + input.value = 30; + input.wantValueStyle = TestValueStyle.Object; // Overflow -> Object + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + input.value = 31; + input.wantValueStyle = TestValueStyle.Overflow; // Object -> Overflow + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + input.value = 32; + input.wantValueStyle = TestValueStyle.Object; // Overflow -> Object again + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Overflow)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + // Object<->Inline conversions + + input.value = 40; + input.wantValueStyle = TestValueStyle.Inline; // Object -> Inline + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + priorSum += input.value; + + input.value = 41; + input.wantValueStyle = TestValueStyle.Object; // Inline -> Object + _ = bContext.RMW(key, ref input, ref output); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Inline)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + + _ = bContext.Read(key, ref input, ref output, Empty.Default); + Assert.That(output.srcValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.destValueStyle, Is.EqualTo(TestValueStyle.Object)); + Assert.That(output.value.value, Is.EqualTo(priorSum + input.value)); + } + + public class TestInlineObjectFunctions : TestObjectFunctions + { + // Force test of overflow values + const int OverflowValueSize = 1 << (LogSettings.kDefaultMaxInlineValueSizeBits + 1); + byte[] pinnedValueOverflowBytes = GC.AllocateArray(OverflowValueSize, pinned: true); + Span GetOverflowValueSpanByte() => new (pinnedValueOverflowBytes); + + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + { + Assert.That(sizeInfo.ValueIsInline, Is.EqualTo(logRecord.Info.ValueIsInline), $"Non-IPU mismatch in sizeInfo ({sizeInfo.ValueIsInline}) and dstLogRecord ({logRecord.Info.ValueIsInline}) ValueIsInline in {Utility.GetCurrentMethodName()}"); + return DoWriter(ref logRecord, ref sizeInfo, ref input, srcValue: null, ref output); + } + + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + { + // Use the same record for source and dest; DoUpdater does not modify dest until all source info is processed. + return DoUpdater(ref logRecord, ref logRecord, ref sizeInfo, input, ref output); + } + + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + { + Assert.That(sizeInfo.ValueIsInline, Is.EqualTo(dstLogRecord.Info.ValueIsInline), $"Non-IPU mismatch in sizeInfo ({sizeInfo.ValueIsInline}) and dstLogRecord ({dstLogRecord.Info.ValueIsInline}) ValueIsInline in {Utility.GetCurrentMethodName()}"); + return DoUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, input, ref output); + } + + private bool DoUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, TestObjectInput input, ref TestObjectOutput output) + where TSourceLogRecord : ISourceLogRecord + { + Set(ref output.srcValueStyle, srcLogRecord.Info); + SetAndVerify(ref input, ref output.destValueStyle, sizeInfo.ValueIsInline, sizeInfo.ValueIsOverflow); + + // If the value is inline it is a ValueStruct; if it is overflow it is a buffer with the first long set to the desired value. + long srcValue; + if (srcLogRecord.Info.ValueIsInline) + srcValue = (int)srcLogRecord.ValueSpan.AsRef().vfield1; + else if (srcLogRecord.Info.ValueIsOverflow) + { + Assert.That(srcLogRecord.ValueSpan.Length, Is.EqualTo(OverflowValueSize)); + srcValue = (int)srcLogRecord.ValueSpan.AsRef(); + } + else + srcValue = ((TestObjectValue)srcLogRecord.ValueObject).value; + + output.value = srcLogRecord.Info.ValueIsObject ? (TestObjectValue)srcLogRecord.ValueObject : new TestObjectValue { value = (int)srcValue }; + output.value.value += input.value; + + 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), ref sizeInfo); + case TestValueStyle.Overflow: + Span overflowValue = GetOverflowValueSpanByte(); + overflowValue.AsRef() = srcValue + input.value; + return logRecord.TrySetValueSpan(overflowValue, ref sizeInfo); + case TestValueStyle.Object: + return logRecord.TrySetValueObject(output.value, ref sizeInfo); + default: + Assert.Fail("Unknown value style"); + return false; + } + } + + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) + { + Set(ref output.srcValueStyle, logRecord.Info); + return DoWriter(ref logRecord, ref sizeInfo, ref input, (TestObjectValue)srcValue, ref output); + } + + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, ref ReadInfo readInfo) + { + Set(ref output.srcValueStyle, srcLogRecord.Info); + + // If the value is inline it is a ValueStruct; if it is overflow it is a buffer with the first long set to the desired value. + if (srcLogRecord.Info.ValueIsInline) + output.value = new TestObjectValue() { value = (int)srcLogRecord.ValueSpan.AsRef().vfield1 }; + else if (srcLogRecord.Info.ValueIsOverflow) + { + Assert.That(srcLogRecord.ValueSpan.Length, Is.EqualTo(OverflowValueSize)); + unsafe { output.value = new TestObjectValue() { value = (int)srcLogRecord.ValueSpan.AsRef() }; } + } + else + output.value = ((TestObjectValue)srcLogRecord.ValueObject); + return true; + } + + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) + { + Assert.That(sizeInfo.ValueIsInline, Is.EqualTo(logRecord.Info.ValueIsInline), $"Non-IPU mismatch in sizeInfo ({sizeInfo.ValueIsInline}) and dstLogRecord ({logRecord.Info.ValueIsInline}) ValueIsInline in {Utility.GetCurrentMethodName()}"); + return DoWriter(ref logRecord, ref sizeInfo, ref input, (TestObjectValue)srcValue, ref output); + } + + private bool DoWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, TestObjectValue srcValue, ref TestObjectOutput output) + { + Assert.That(srcValue, Is.Null, "srcValue should be null for these upsert tests; use Input instead"); + output.srcValueStyle = TestValueStyle.None; + SetAndVerify(ref input, ref output.destValueStyle, sizeInfo.ValueIsInline, sizeInfo.ValueIsOverflow); + + output.value = new TestObjectValue { value = input.value }; + switch (output.destValueStyle) + { + case TestValueStyle.Inline: + ValueStruct valueStruct = new() { vfield1 = input.value, vfield2 = input.value * 100 }; + return logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref valueStruct), ref sizeInfo); + case TestValueStyle.Overflow: + Span overflowValue = GetOverflowValueSpanByte(); + overflowValue.AsRef() = input.value; + return logRecord.TrySetValueSpan(overflowValue, ref sizeInfo); + case TestValueStyle.Object: + return logRecord.TrySetValueObject(output.value, ref sizeInfo); + default: + Assert.Fail("Unknown value style"); + return false; + } + } + + static void Set(ref TestValueStyle style, RecordInfo recordInfo) => Set(ref style, recordInfo.ValueIsInline, recordInfo.ValueIsOverflow); + + static void Set(ref TestValueStyle style, bool isInline, bool isOverflow) + { + style = (isInline, isOverflow) switch + { + (true, false) => TestValueStyle.Inline, + (false, true) => TestValueStyle.Overflow, + _ => TestValueStyle.Object + }; + } + static void SetAndVerify(ref TestObjectInput input, ref TestValueStyle style, bool isInline, bool isOverflow) + { + Set(ref style, isInline, isOverflow); + Assert.That(input.wantValueStyle, Is.EqualTo(style)); + } + + static RecordFieldInfo GetFieldInfo(ReadOnlySpan key, ref TestObjectInput input) + => new() + { + KeyDataSize = key.Length, + ValueDataSize = input.wantValueStyle switch + { + TestValueStyle.Inline => ValueStruct.AsSpanByteDataSize, + TestValueStyle.Overflow => OverflowValueSize, + TestValueStyle.Object => ObjectIdMap.ObjectIdSize, + _ => int.MaxValue + }, + ValueIsObject = input.wantValueStyle == TestValueStyle.Object + }; + + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input) + => GetFieldInfo(srcLogRecord.Key, ref input); + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) + => GetFieldInfo(key, ref input); + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref TestObjectInput input) + => GetFieldInfo(key, ref input); + } + } +} diff --git a/libs/storage/Tsavorite/cs/test/ObjectReadCacheTests.cs b/libs/storage/Tsavorite/cs/test/ObjectReadCacheTests.cs index 6b7ef35822a..744d6d1a058 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectReadCacheTests.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectReadCacheTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -227,4 +229,6 @@ public void ObjectDiskWriteReadCache2() } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest.cs b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest.cs index 75b59cda5f4..9ddb665f36a 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Threading.Tasks; @@ -167,4 +169,6 @@ public unsafe void Verify(Guid cprVersion, Guid indexVersion) session.Dispose(); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest2.cs b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest2.cs index 2a528c61293..15561cfa271 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest2.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest2.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using System.Threading.Tasks; using NUnit.Framework; @@ -241,21 +243,15 @@ public override bool InPlaceUpdater(ref MyKey key, ref MyInput input, ref MyValu } - public override bool SingleReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) + public override bool Reader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) { dst.value = value; return true; } - public override bool SingleWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) { dst = src; return true; } + public override bool InitialWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { dst = src; return true; } - public override bool ConcurrentReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - dst.value = value; - return true; - } - - public override bool ConcurrentWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { if (src == null) return false; @@ -269,4 +265,6 @@ public override bool ConcurrentWriter(ref MyKey key, ref MyInput input, ref MyVa public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, ref MyOutput output, MyContext ctx, Status status, RecordMetadata recordMetadata) => ctx.Populate(ref status, ref output); } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest3.cs b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest3.cs index ef716e59722..fe8c038c5b6 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest3.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectRecoveryTest3.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -157,4 +159,6 @@ private static void Read(ClientSession { // Read functions - public override bool SingleReader(ref AdIdObj key, ref Input input, ref NumClicksObj value, ref Output dst, ref ReadInfo readInfo) - { - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref AdIdObj key, ref Input input, ref NumClicksObj value, ref Output dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref AdIdObj key, ref Input input, ref NumClicksObj value, ref Output dst, ref ReadInfo readInfo) { dst.value = value; return true; @@ -84,4 +80,6 @@ public override bool CopyUpdater(ref AdIdObj key, ref Input input, ref NumClicks return true; } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs b/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs index 83d746e844a..429869ec846 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectTestTypes.cs @@ -1,234 +1,144 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using NUnit.Framework.Legacy; using Tsavorite.core; namespace Tsavorite.test { - public class MyKey + public enum TestValueStyle : byte { None, Inline, Overflow, Object }; + + public struct TestObjectKey { public int key; public override string ToString() => key.ToString(); - public struct Comparer : IKeyComparer + public struct Comparer : IKeyComparer { - public long GetHashCode64(ref MyKey key) => Utility.GetHashCode(key.key); + public readonly long GetHashCode64(ReadOnlySpan key) => Utility.GetHashCode(key.AsRef().key); - public bool Equals(ref MyKey k1, ref MyKey k2) => k1.key == k2.key; + public readonly bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef().key == k2.AsRef().key; } } - public class MyKeySerializer : BinaryObjectSerializer + public class TestObjectValue : IHeapObject { - public override void Deserialize(out MyKey obj) => obj = new MyKey { key = reader.ReadInt32() }; + public int value; - public override void Serialize(ref MyKey obj) => writer.Write(obj.key); - } + 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 class MyValue - { - public int value; + public void Dispose() { } public override string ToString() => value.ToString(); - public struct Comparer : IKeyComparer // This Value comparer is used by a test + public class Serializer : BinaryObjectSerializer { - public long GetHashCode64(ref MyValue k) => Utility.GetHashCode(k.value); + public override void Deserialize(out IHeapObject obj) => obj = new TestObjectValue { value = reader.ReadInt32() }; - public bool Equals(ref MyValue k1, ref MyValue k2) => k1.value == k2.value; + public override void Serialize(IHeapObject obj) => writer.Write(((TestObjectValue)obj).value); } } - public class MyValueSerializer : BinaryObjectSerializer - { - public override void Deserialize(out MyValue obj) => obj = new MyValue { value = reader.ReadInt32() }; - - public override void Serialize(ref MyValue obj) => writer.Write(obj.value); - } - - public class MyInput + public struct TestObjectInput { public int value; - public override string ToString() => value.ToString(); - } - - public class MyOutput - { - public MyValue value; + public TestValueStyle wantValueStyle; - public override string ToString() => value.ToString(); + public override readonly string ToString() => $"value {value}, wantValStyle {wantValueStyle}"; } - public class MyFunctions : SessionFunctionsBase + public struct TestObjectOutput { - public override bool InitialUpdater(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value = new MyValue { value = input.value }; - return true; - } - - public override bool InPlaceUpdater(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value.value += input.value; - return true; - } - - public override bool NeedCopyUpdate(ref MyKey key, ref MyInput input, ref MyValue oldValue, ref MyOutput output, ref RMWInfo rmwInfo) => true; - - public override bool CopyUpdater(ref MyKey key, ref MyInput input, ref MyValue oldValue, ref MyValue newValue, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue = new MyValue { value = oldValue.value + input.value }; - return true; - } - - public override bool ConcurrentReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - if (dst == default) - dst = new MyOutput(); - dst.value = value; - return true; - } - - public override bool ConcurrentWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - dst.value = src.value; - return true; - } - - public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, ref MyOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) - { - ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(output.value.value, key.key); - } - - public override void RMWCompletionCallback(ref MyKey key, ref MyInput input, ref MyOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) - { - ClassicAssert.IsTrue(status.Found); - ClassicAssert.IsTrue(status.Record.CopyUpdated); - } + public TestObjectValue value; - public override bool SingleReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) - { - if (dst == default) - dst = new MyOutput(); - dst.value = value; - return true; - } + public TestValueStyle srcValueStyle; + public TestValueStyle destValueStyle; - public override bool SingleWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } + public override readonly string ToString() => $"value {value}, srcValStyle {srcValueStyle}, destValStyle {destValueStyle}"; } - public class MyFunctions2 : SessionFunctionsBase + public class TestObjectFunctions : SessionFunctionsBase { - public override bool InitialUpdater(ref MyValue key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value = new MyValue { value = input.value }; - return true; - } + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + => logRecord.TrySetValueObject(new TestObjectValue { value = input.value }, ref sizeInfo); - public override bool InPlaceUpdater(ref MyValue key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) { - value.value += input.value; + ((TestObjectValue)logRecord.ValueObject).value += input.value; return true; } - public override bool NeedCopyUpdate(ref MyValue key, ref MyInput input, ref MyValue oldValue, ref MyOutput output, ref RMWInfo rmwInfo) => true; + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) => true; - public override bool CopyUpdater(ref MyValue key, ref MyInput input, ref MyValue oldValue, ref MyValue newValue, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue = new MyValue { value = oldValue.value + input.value }; - return true; - } + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + => dstLogRecord.TrySetValueObject(new TestObjectValue { value = ((TestObjectValue)srcLogRecord.ValueObject).value + input.value }, ref sizeInfo); - public override bool ConcurrentReader(ref MyValue key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) { - if (dst == default) - dst = new MyOutput(); - dst.value = value; + if (!logRecord.TrySetValueObject(srcValue, ref sizeInfo)) + return false; + output.value = (TestObjectValue)logRecord.ValueObject; return true; } - public override bool ConcurrentWriter(ref MyValue key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - dst.value = src.value; - return true; - } - - public override void ReadCompletionCallback(ref MyValue key, ref MyInput input, ref MyOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(key.value, output.value.value); + ClassicAssert.AreEqual(output.value.value, srcLogRecord.Key.AsRef().key); } - public override void RMWCompletionCallback(ref MyValue key, ref MyInput input, ref MyOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); ClassicAssert.IsTrue(status.Record.CopyUpdated); } - public override bool SingleReader(ref MyValue key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, ref ReadInfo readInfo) { - if (dst == default) - dst = new MyOutput(); - dst.value = value; + output.value = (TestObjectValue)srcLogRecord.ValueObject; return true; } - public override bool SingleWriter(ref MyValue key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) + => logRecord.TrySetValueObject(srcValue, ref sizeInfo); + + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) + => new() { KeyDataSize = key.Length, ValueDataSize = 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 }; } - public class MyFunctionsDelete : SessionFunctionsBase + public class TestObjectFunctionsDelete : SessionFunctionsBase { - public override bool InitialUpdater(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value = new MyValue { value = input.value }; - return true; - } + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + => logRecord.TrySetValueObject(new TestObjectValue { value = input.value }, ref sizeInfo); - public override bool InPlaceUpdater(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) { - value.value += input.value; + ((TestObjectValue)logRecord.ValueObject).value += input.value; return true; } - public override bool NeedCopyUpdate(ref MyKey key, ref MyInput input, ref MyValue oldValue, ref MyOutput output, ref RMWInfo rmwInfo) => true; - - public override bool CopyUpdater(ref MyKey key, ref MyInput input, ref MyValue oldValue, ref MyValue newValue, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue = new MyValue { value = oldValue.value + input.value }; - return true; - } + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) => true; - public override bool ConcurrentReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - dst ??= new MyOutput(); - dst.value = value; - return true; - } + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, ref TestObjectOutput output, ref RMWInfo rmwInfo) + => dstLogRecord.TrySetValueObject(new TestObjectValue { value = ((TestObjectValue)srcLogRecord.ValueObject).value + input.value }, ref sizeInfo); - public override bool ConcurrentWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - dst = src; - return true; - } + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) + => logRecord.TrySetValueObject(srcValue, ref sizeInfo); - public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, ref MyOutput output, int ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, int ctx, Status status, RecordMetadata recordMetadata) { if (ctx == 0) { ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(key.key, output.value.value); + ClassicAssert.AreEqual(srcLogRecord.Key.AsRef().key, output.value.value); } else if (ctx == 1) { @@ -236,7 +146,7 @@ public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, re } } - public override void RMWCompletionCallback(ref MyKey key, ref MyInput input, ref MyOutput output, int ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, int ctx, Status status, RecordMetadata recordMetadata) { if (ctx == 0) { @@ -247,109 +157,70 @@ public override void RMWCompletionCallback(ref MyKey key, ref MyInput input, ref ClassicAssert.IsFalse(status.Found); } - public override bool SingleReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) - { - dst ??= new MyOutput(); - dst.value = value; - return true; - } - - public override bool SingleWriter(ref MyKey key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } - } - - public class MixedFunctions : SessionFunctionsBase - { - public override bool InitialUpdater(ref int key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value = new MyValue { value = input.value }; - return true; - } - - public override bool InPlaceUpdater(ref int key, ref MyInput input, ref MyValue value, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value.value += input.value; - return true; - } - - public override bool NeedCopyUpdate(ref int key, ref MyInput input, ref MyValue oldValue, ref MyOutput output, ref RMWInfo rmwInfo) => true; - - public override bool CopyUpdater(ref int key, ref MyInput input, ref MyValue oldValue, ref MyValue newValue, ref MyOutput output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue = new MyValue { value = oldValue.value + input.value }; - return true; - } - - public override bool ConcurrentReader(ref int key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - dst.value = value; - return true; - } - - public override bool ConcurrentWriter(ref int key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo updateInfo, ref RecordInfo recordInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestObjectOutput output, ref ReadInfo readInfo) { - dst.value = src.value; + output.value = (TestObjectValue)srcLogRecord.ValueObject; return true; } - public override bool SingleReader(ref int key, ref MyInput input, ref MyValue value, ref MyOutput dst, ref ReadInfo readInfo) - { - dst.value = value; - return true; - } + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestObjectOutput output, ref UpsertInfo upsertInfo) + => logRecord.TrySetValueObject(srcValue, ref sizeInfo); - public override bool SingleWriter(ref int key, ref MyInput input, ref MyValue src, ref MyValue dst, ref MyOutput output, ref UpsertInfo updateInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref TestObjectInput input) + => new() { KeyDataSize = key.Length, ValueDataSize = 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 }; } - public class MyLargeValue + public class TestLargeObjectValue : IHeapObject { public byte[] value; - public MyLargeValue() + 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 void Dispose() { } + + public TestLargeObjectValue() { } - public MyLargeValue(int size) + public TestLargeObjectValue(int size) { value = new byte[size]; for (int i = 0; i < size; i++) - { value[i] = (byte)(size + i); - } } - } - public class MyLargeValueSerializer : BinaryObjectSerializer - { - public override void Deserialize(out MyLargeValue obj) + public class Serializer : BinaryObjectSerializer { - obj = new MyLargeValue(); - int size = reader.ReadInt32(); - obj.value = reader.ReadBytes(size); - } + public override void Deserialize(out IHeapObject obj) + { + var value = new TestLargeObjectValue(); + obj = value; + int size = reader.ReadInt32(); + value.value = reader.ReadBytes(size); + } - public override void Serialize(ref MyLargeValue obj) - { - writer.Write(obj.value.Length); - writer.Write(obj.value); + public override void Serialize(IHeapObject obj) + { + var value = (TestLargeObjectValue)obj; + writer.Write(value.value.Length); + writer.Write(value.value); + } } } - public class MyLargeOutput + public class TestLargeObjectOutput { - public MyLargeValue value; + public TestLargeObjectValue value; } - public class MyLargeFunctions : SessionFunctionsBase + public class TestLargeObjectFunctions : SessionFunctionsBase { - public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, ref MyLargeOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord srcLogRecord, ref TestObjectInput input, ref TestLargeObjectOutput output, Empty ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); for (int i = 0; i < output.value.value.Length; i++) @@ -358,28 +229,36 @@ public override void ReadCompletionCallback(ref MyKey key, ref MyInput input, re } } - public override bool SingleReader(ref MyKey key, ref MyInput input, ref MyLargeValue value, ref MyLargeOutput dst, ref ReadInfo readInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref TestObjectInput input, ref TestLargeObjectOutput output, ref ReadInfo readInfo) { - dst.value = value; + output.value = (TestLargeObjectValue)srcLogRecord.ValueObject; return true; } - public override bool ConcurrentReader(ref MyKey key, ref MyInput input, ref MyLargeValue value, ref MyLargeOutput dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) { - dst.value = value; + if (!logRecord.TrySetValueObject(srcValue)) // We should always be non-inline + return false; + output.value = (TestLargeObjectValue)logRecord.ValueObject; return true; } - public override bool ConcurrentWriter(ref MyKey key, ref MyInput input, ref MyLargeValue src, ref MyLargeValue dst, ref MyLargeOutput output, ref UpsertInfo updateInfo, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref TestObjectInput input, IHeapObject srcValue, ref TestLargeObjectOutput output, ref UpsertInfo updateInfo) { - dst = src; + if (!logRecord.TrySetValueObject(srcValue)) // We should always be non-inline + return false; + output.value = (TestLargeObjectValue)logRecord.ValueObject; return true; } - public override bool SingleWriter(ref MyKey key, ref MyInput input, ref MyLargeValue src, ref MyLargeValue dst, ref MyLargeOutput output, ref UpsertInfo updateInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = src; - return true; - } + public override RecordFieldInfo GetRMWModifiedFieldInfo(ref 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, ref TSourceLogRecord inputLogRecord, ref TestObjectInput input) + => new() { KeyDataSize = key.Length, ValueDataSize = inputLogRecord.Info.ValueIsObject ? ObjectIdMap.ObjectIdSize : inputLogRecord.ValueSpan.Length, ValueIsObject = inputLogRecord.Info.ValueIsObject, + HasETag = inputLogRecord.Info.HasETag, HasExpiration = inputLogRecord.Info.HasExpiration}; } -} \ No newline at end of file +} diff --git a/libs/storage/Tsavorite/cs/test/ObjectTests.cs b/libs/storage/Tsavorite/cs/test/ObjectTests.cs index b0bba31558e..8930d53ad0b 100644 --- a/libs/storage/Tsavorite/cs/test/ObjectTests.cs +++ b/libs/storage/Tsavorite/cs/test/ObjectTests.cs @@ -9,13 +9,13 @@ namespace Tsavorite.test { - using ClassAllocator = GenericAllocator>>; - using ClassStoreFunctions = StoreFunctions>; + using ClassAllocator = ObjectAllocator>; + using ClassStoreFunctions = StoreFunctions; [TestFixture] internal class ObjectTests { - private TsavoriteKV store; + private TsavoriteKV store; private IDevice log, objlog; [SetUp] @@ -33,7 +33,7 @@ public void Setup() MutableFraction = 0.1, MemorySize = 1L << 15, PageSize = 1L << 10 - }, StoreFunctions.Create(new MyKey.Comparer(), () => new MyKeySerializer(), () => new MyValueSerializer(), DefaultRecordDisposer.Instance) + }, StoreFunctions.Create(new TestObjectKey.Comparer(), () => new TestObjectValue.Serializer(), DefaultRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -50,72 +50,70 @@ public void TearDown() DeleteDirectory(MethodTestDir); } - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] + [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] public void ObjectInMemWriteRead() { - using var session = store.NewSession(new MyFunctions()); + using var session = store.NewSession(new TestObjectFunctions()); var bContext = session.BasicContext; - MyKey key1 = new() { key = 9999999 }; - MyValue value = new() { value = 23 }; + TestObjectKey keyStruct = new() { key = 9999999 }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + TestObjectValue value = new() { value = 23 }; - MyInput input = null; - MyOutput output = new(); + TestObjectInput input = default; + TestObjectOutput output = default; - _ = bContext.Upsert(ref key1, ref value, Empty.Default); - _ = bContext.Read(ref key1, ref input, ref output, Empty.Default); + _ = bContext.Upsert(key, value, Empty.Default); + _ = bContext.Read(key, ref input, ref output, Empty.Default); ClassicAssert.AreEqual(value.value, output.value.value); } - [Test] - [Category("TsavoriteKV")] + [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] public void ObjectInMemWriteRead2() { - using var session = store.NewSession(new MyFunctions()); + using var session = store.NewSession(new TestObjectFunctions()); var bContext = session.BasicContext; - MyKey key1 = new() { key = 8999998 }; - MyInput input1 = new() { value = 23 }; - MyOutput output = new(); + TestObjectKey key1Struct = new() { key = 8999998 }; + var key1 = SpanByte.FromPinnedVariable(ref key1Struct); + TestObjectInput input1 = new() { value = 23 }; + TestObjectOutput output = new(); - _ = bContext.RMW(ref key1, ref input1, Empty.Default); + _ = bContext.RMW(key1, ref input1, Empty.Default); - MyKey key2 = new() { key = 8999999 }; - MyInput input2 = new() { value = 24 }; - _ = bContext.RMW(ref key2, ref input2, Empty.Default); + TestObjectKey key2Struct = new() { key = 8999999 }; + var key2 = SpanByte.FromPinnedVariable(ref key2Struct); + TestObjectInput input2 = new() { value = 24 }; + _ = bContext.RMW(key2, ref input2, Empty.Default); - _ = bContext.Read(ref key1, ref input1, ref output, Empty.Default); + _ = bContext.Read(key1, ref input1, ref output, Empty.Default); ClassicAssert.AreEqual(input1.value, output.value.value); - _ = bContext.Read(ref key2, ref input2, ref output, Empty.Default); + _ = bContext.Read(key2, ref input2, ref output, Empty.Default); ClassicAssert.AreEqual(input2.value, output.value.value); - } - - [Test] - [Category("TsavoriteKV")] - [Category("Smoke")] + [Test, Category(TsavoriteKVTestCategory), Category(SmokeTestCategory), Category(ObjectIdMapCategory)] public void ObjectDiskWriteRead() { - using var session = store.NewSession(new MyFunctions()); + using var session = store.NewSession(new TestObjectFunctions()); var bContext = session.BasicContext; for (int i = 0; i < 2000; i++) { - var key = new MyKey { key = i }; - var value = new MyValue { value = i }; - _ = bContext.Upsert(ref key, ref value, Empty.Default); + var key1Struct = new TestObjectKey { key = i }; + var key = SpanByte.FromPinnedVariable(ref key1Struct); + var value = new TestObjectValue { value = i }; + _ = bContext.Upsert(key, value, Empty.Default); // store.ShiftReadOnlyAddress(store.LogTailAddress); } - MyKey key2 = new() { key = 23 }; - MyInput input = new(); - MyOutput g1 = new(); - var status = bContext.Read(ref key2, ref input, ref g1, Empty.Default); + TestObjectKey key2Struct = new() { key = 23 }; + var key2 = SpanByte.FromPinnedVariable(ref key2Struct); + TestObjectInput input = new(); + TestObjectOutput g1 = new(); + var status = bContext.Read(key2, ref input, ref g1, Empty.Default); if (status.IsPending) { @@ -126,8 +124,8 @@ public void ObjectDiskWriteRead() ClassicAssert.IsTrue(status.Found); ClassicAssert.AreEqual(23, g1.value.value); - key2 = new MyKey { key = 99999 }; - status = bContext.Read(ref key2, ref input, ref g1, Empty.Default); + key2Struct.key = 99999; + status = bContext.Read(key2, ref input, ref g1, Empty.Default); if (status.IsPending) (status, _) = bContext.GetSinglePendingResult(); @@ -136,29 +134,32 @@ public void ObjectDiskWriteRead() // Update last 100 using RMW in memory for (int i = 1900; i < 2000; i++) { - var key = new MyKey { key = i }; - input = new MyInput { value = 1 }; - status = bContext.RMW(ref key, ref input, Empty.Default); + var keyStruct = new TestObjectKey { key = i }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + input = new TestObjectInput { value = 1 }; + status = bContext.RMW(key, ref input, Empty.Default); ClassicAssert.IsFalse(status.IsPending, "Expected RMW to complete in-memory"); } // Update first 100 using RMW from storage for (int i = 0; i < 100; i++) { - var key1 = new MyKey { key = i }; - input = new MyInput { value = 1 }; - status = bContext.RMW(ref key1, ref input, Empty.Default); + var keyStruct = new TestObjectKey { key = i }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + input = new TestObjectInput { value = 1 }; + status = bContext.RMW(key, ref input, Empty.Default); if (status.IsPending) _ = bContext.CompletePending(true); } for (int i = 0; i < 2000; i++) { - var output = new MyOutput(); - var key1 = new MyKey { key = i }; - var value = new MyValue { value = i }; + var output = new TestObjectOutput(); + var keyStruct = new TestObjectKey { key = i }; + var key = SpanByte.FromPinnedVariable(ref keyStruct); + var value = new TestObjectValue { value = i }; - status = bContext.Read(ref key1, ref input, ref output, Empty.Default); + status = bContext.Read(key, ref input, ref output, Empty.Default); if (status.IsPending) (status, output) = bContext.GetSinglePendingResult(); else @@ -171,4 +172,4 @@ public void ObjectDiskWriteRead() } } } -} \ No newline at end of file +} diff --git a/libs/storage/Tsavorite/cs/test/OverflowBucketLockTableTests.cs b/libs/storage/Tsavorite/cs/test/OverflowBucketLockTableTests.cs index 4cb0324b697..a41f7df1fe9 100644 --- a/libs/storage/Tsavorite/cs/test/OverflowBucketLockTableTests.cs +++ b/libs/storage/Tsavorite/cs/test/OverflowBucketLockTableTests.cs @@ -13,13 +13,13 @@ namespace Tsavorite.test.LockTable { - using LongStoreFunctions = StoreFunctions>; + using LongStoreFunctions = StoreFunctions; - internal class SingleBucketComparer : IKeyComparer + internal class SingleBucketComparer : IKeyComparer { - public bool Equals(ref long k1, ref long k2) => k1 == k2; + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef() == k2.AsRef(); - public long GetHashCode64(ref long k) => 42L; + public long GetHashCode64(ReadOnlySpan k) => 42L; } // Used to signal Setup to use the SingleBucketComparer @@ -28,11 +28,11 @@ public enum UseSingleBucketComparer { UseSingleBucket } [TestFixture] internal class OverflowBucketLockTableTests { - IKeyComparer comparer = new LongKeyComparer(); + IKeyComparer comparer = new LongKeyComparer(); long SingleBucketKey = 1; // We use a single bucket here for most tests so this lets us use 'ref' easily // For OverflowBucketLockTable, we need an instance of TsavoriteKV - private TsavoriteKV> store; + private TsavoriteKV> store; private IDevice log; [SetUp] @@ -58,7 +58,7 @@ public void Setup() LogDevice = log, PageSize = 1L << 12, MemorySize = 1L << 22 - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -76,7 +76,7 @@ public void TearDown() void TryLock(long key, LockType lockType, int expectedCurrentReadLocks, bool expectedLockResult) { - HashEntryInfo hei = new(comparer.GetHashCode64(ref key)); + HashEntryInfo hei = new(comparer.GetHashCode64(SpanByte.FromPinnedVariable(ref key))); PopulateHei(ref hei); // Check for existing lock @@ -91,7 +91,7 @@ void TryLock(long key, LockType lockType, int expectedCurrentReadLocks, bool exp void Unlock(long key, LockType lockType) { - HashEntryInfo hei = new(comparer.GetHashCode64(ref key)); + HashEntryInfo hei = new(comparer.GetHashCode64(SpanByte.FromPinnedVariable(ref key))); PopulateHei(ref hei); if (lockType == LockType.Shared) store.LockTable.UnlockShared(ref hei); @@ -101,9 +101,9 @@ void Unlock(long key, LockType lockType) internal void PopulateHei(ref HashEntryInfo hei) => PopulateHei(store, ref hei); - internal static void PopulateHei(TsavoriteKV store, ref HashEntryInfo hei) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static void PopulateHei(TsavoriteKV store, ref HashEntryInfo hei) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => store.FindOrCreateTag(ref hei, store.Log.BeginAddress); internal void AssertLockCounts(ref HashEntryInfo hei, bool expectedX, long expectedS) @@ -113,42 +113,22 @@ internal void AssertLockCounts(ref HashEntryInfo hei, bool expectedX, long expec ClassicAssert.AreEqual(expectedS, lockState.NumLockedShared); } - internal static void AssertLockCounts(TsavoriteKV store, TKey key, bool expectedX, int expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - => AssertLockCounts(store, ref key, expectedX, expectedS); - - internal static void AssertLockCounts(TsavoriteKV store, ref TKey key, bool expectedX, int expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static void AssertLockCounts(TsavoriteKV store, ReadOnlySpan key, bool expectedX, int expectedS) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(ref key)); + HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(key)); PopulateHei(store, ref hei); var lockState = store.LockTable.GetLockState(ref hei); ClassicAssert.AreEqual(expectedX, lockState.IsLockedExclusive, "XLock mismatch"); ClassicAssert.AreEqual(expectedS, lockState.NumLockedShared, "SLock mismatch"); } - internal static void AssertLockCounts(TsavoriteKV store, ref TKey key, bool expectedX, bool expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static void AssertLockCounts(TsavoriteKV store, ref FixedLengthTransactionalKeyStruct keyStruct, bool expectedX, bool expectedS) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - FixedLengthLockableKeyStruct keyStruct = new() - { - Key = key, - KeyHash = store.storeFunctions.GetKeyHashCode64(ref key), - LockType = LockType.None, // Not used for this call - }; - keyStruct.KeyHash = store.GetKeyHash(ref key); - AssertLockCounts(store, ref keyStruct, expectedX, expectedS); - } - - - internal static void AssertLockCounts(TsavoriteKV store, ref FixedLengthLockableKeyStruct key, bool expectedX, bool expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - HashEntryInfo hei = new(key.KeyHash); + HashEntryInfo hei = new(keyStruct.KeyHash); PopulateHei(store, ref hei); var lockState = store.LockTable.GetLockState(ref hei); ClassicAssert.AreEqual(expectedX, lockState.IsLockedExclusive, "XLock mismatch"); @@ -158,9 +138,9 @@ internal static void AssertLockCounts internal unsafe void AssertTotalLockCounts(long expectedX, long expectedS) => AssertTotalLockCounts(store, expectedX, expectedS); - internal static unsafe void AssertTotalLockCounts(TsavoriteKV store, long expectedX, long expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe void AssertTotalLockCounts(TsavoriteKV store, long expectedX, long expectedS) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { HashBucket* buckets = store.state[store.resizeInfo.version].tableAligned; var count = store.LockTable.NumBuckets; @@ -175,11 +155,11 @@ internal static unsafe void AssertTotalLockCounts(T ClassicAssert.AreEqual(expectedS, scount); } - internal void AssertBucketLockCount(ref FixedLengthLockableKeyStruct key, long expectedX, long expectedS) => AssertBucketLockCount(store, ref key, expectedX, expectedS); + internal void AssertBucketLockCount(ref FixedLengthTransactionalKeyStruct keyStruct, long expectedX, long expectedS) => AssertBucketLockCount(store, ref keyStruct, expectedX, expectedS); - internal static unsafe void AssertBucketLockCount(TsavoriteKV store, ref FixedLengthLockableKeyStruct key, long expectedX, long expectedS) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe void AssertBucketLockCount(TsavoriteKV store, ref FixedLengthTransactionalKeyStruct key, long expectedX, long expectedS) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { var bucketIndex = store.LockTable.GetBucketIndex(key.KeyHash); var bucket = store.state[store.resizeInfo.version].tableAligned + bucketIndex; @@ -191,7 +171,7 @@ internal static unsafe void AssertBucketLockCount(T [Category(LockTestCategory), Category(LockTableTestCategory), Category(SmokeTestCategory)] public void SingleKeyTest([Values] UseSingleBucketComparer /* justToSignalSetup */ _) { - HashEntryInfo hei = new(comparer.GetHashCode64(ref SingleBucketKey)); + HashEntryInfo hei = new(comparer.GetHashCode64(SpanByte.FromPinnedVariable(ref SingleBucketKey))); PopulateHei(ref hei); AssertLockCounts(ref hei, false, 0); @@ -227,7 +207,7 @@ public void SingleKeyTest([Values] UseSingleBucketComparer /* justToSignalSetup [Category(LockTestCategory), Category(LockTableTestCategory), Category(SmokeTestCategory)] public void ThreeKeyTest([Values] UseSingleBucketComparer /* justToSignalSetup */ _) { - HashEntryInfo hei = new(comparer.GetHashCode64(ref SingleBucketKey)); + HashEntryInfo hei = new(comparer.GetHashCode64(SpanByte.FromPinnedVariable(ref SingleBucketKey))); PopulateHei(ref hei); AssertLockCounts(ref hei, false, 0); @@ -303,15 +283,16 @@ public void ThreadedLockStressTestMultiThreadsRandomContention([Values(3, 8)] in AssertTotalLockCounts(0, 0); } - FixedLengthLockableKeyStruct[] CreateKeys(Random rng, int numKeys, int numRecords) + FixedLengthTransactionalKeyStruct[] CreateKeys(Random rng, int numKeys, int numRecords) { - FixedLengthLockableKeyStruct createKey() + FixedLengthTransactionalKeyStruct createKey() { - long key = rng.Next(numKeys); - var keyHash = store.GetKeyHash(ref key); + long keyLong = rng.Next(numKeys); + var key = SpanByte.FromPinnedVariable(ref keyLong); + var keyHash = store.GetKeyHash(key); return new() { - Key = key, + Key = PinnedSpanByte.FromPinnedSpan(key), // LockType.None means split randomly between Shared and Exclusive LockType = rng.Next(0, 100) < 25 ? LockType.Exclusive : LockType.Shared, KeyHash = keyHash, @@ -320,7 +301,7 @@ FixedLengthLockableKeyStruct createKey() return [.. Enumerable.Range(0, numRecords).Select(ii => createKey())]; } - void AssertSorted(FixedLengthLockableKeyStruct[] keys, int count) + void AssertSorted(FixedLengthTransactionalKeyStruct[] keys, int count) { long prevCode = default; long lastXcode = default; @@ -394,16 +375,17 @@ void runThread(int tid) { Random rng = new(101 * tid); - // maxNumKeys < 0 means use random number of keys + // maxNumKeys < 0 means use random number of keys. SpanByte requires persistent storage so we need the threadKeyNums vector in parallel with threadStructs. int numKeys = maxNumKeys < 0 ? rng.Next(1, -maxNumKeys) : maxNumKeys; - var threadStructs = new FixedLengthLockableKeyStruct[numKeys]; + var threadKeyNums = new long[numKeys]; + var threadStructs = new FixedLengthTransactionalKeyStruct[numKeys]; long getNextKey() { while (true) { var key = rng.Next(lowKey, highKey + 1); // +1 because the end # is not included - if (!Array.Exists(threadStructs, it => it.Key == key)) + if (!Array.Exists(threadStructs, it => it.Key.Length > 0 && it.Key.ReadOnlySpan.AsRef() == key)) return key; } } @@ -413,15 +395,17 @@ long getNextKey() // Create key structs for (var ii = 0; ii < numKeys; ++ii) { - var key = getNextKey(); - threadStructs[ii] = new() // local var for debugging + var keyNum = getNextKey(); + threadKeyNums[ii] = keyNum; + var key = SpanByte.FromPinnedVariable(ref threadKeyNums[ii]); // storage for the SpanByte + threadStructs[ii] = new() // local var for debugging { - Key = key, + Key = PinnedSpanByte.FromPinnedSpan(key), // LockType.None means split randomly between Shared and Exclusive LockType = lockType == LockType.None ? (rng.Next(0, 100) > 50 ? LockType.Shared : LockType.Exclusive) : lockType, - KeyHash = comparer.GetHashCode64(ref key), + KeyHash = comparer.GetHashCode64(key), }; - threadStructs[ii].KeyHash = store.GetKeyHash(ref key); + threadStructs[ii].KeyHash = store.GetKeyHash(key); } // Sort and lock @@ -451,7 +435,6 @@ long getNextKey() } Array.Clear(threadStructs); } - } for (int t = 1; t <= numThreads; t++) diff --git a/libs/storage/Tsavorite/cs/test/PostOperationsTests.cs b/libs/storage/Tsavorite/cs/test/PostOperationsTests.cs index c6d8a7982ed..903c38a1edb 100644 --- a/libs/storage/Tsavorite/cs/test/PostOperationsTests.cs +++ b/libs/storage/Tsavorite/cs/test/PostOperationsTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using NUnit.Framework; using NUnit.Framework.Legacy; @@ -15,7 +17,7 @@ namespace Tsavorite.test [TestFixture] internal class PostOperationsTests { - class PostFunctions : SimpleSimpleFunctions + class PostFunctions : SimpleLongSimpleFunctions { internal long pswAddress; internal long piuAddress; @@ -33,7 +35,7 @@ internal void Clear() internal PostFunctions() : base() { } - public override void PostSingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason) { pswAddress = upsertInfo.Address; } + public override void PostInitialWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo) { pswAddress = upsertInfo.Address; } public override bool InitialUpdater(ref int key, ref int input, ref int value, ref int output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) { value = input; return true; } /// @@ -52,8 +54,8 @@ public override bool PostCopyUpdater(ref int key, ref int input, ref int oldValu return !returnFalseFromPCU; } - public override void PostSingleDeleter(ref int key, ref DeleteInfo deleteInfo) { psdAddress = deleteInfo.Address; } - public override bool ConcurrentDeleter(ref int key, ref int value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => false; + public override void PostInitialDeleter(ref int key, ref DeleteInfo deleteInfo) { psdAddress = deleteInfo.Address; } + public override bool InPlaceDeleter(ref int key, ref int value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) => false; } private TsavoriteKV store; @@ -122,7 +124,7 @@ internal void CompletePendingAndVerifyInsertedAddress() [Test] [Category("TsavoriteKV")] [Category("Smoke")] - public void PostSingleWriterTest() + public void PostInitialWriterTest() { // Populate has already executed the not-found test (InternalInsert) as part of its normal insert. @@ -203,9 +205,9 @@ public void PostCopyUpdaterFalseTest([Values(FlushMode.ReadOnly, FlushMode.OnDis [Test] [Category("TsavoriteKV")] [Category("Smoke")] - public void PostSingleDeleterTest() + public void PostInitialDeleterTest() { - // Execute the not-in-memory test (InternalDelete); ConcurrentDeleter returns false to force a new record to be added. + // Execute the not-in-memory test (InternalDelete); InPlaceDeleter returns false to force a new record to be added. _ = bContext.Delete(TargetKey); ClassicAssert.AreEqual(expectedAddress, session.functions.psdAddress); @@ -216,4 +218,6 @@ public void PostSingleDeleterTest() ClassicAssert.AreEqual(expectedAddress, session.functions.psdAddress); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ReadAddressTests.cs b/libs/storage/Tsavorite/cs/test/ReadAddressTests.cs index 09041311a8e..9ae6f6e4f11 100644 --- a/libs/storage/Tsavorite/cs/test/ReadAddressTests.cs +++ b/libs/storage/Tsavorite/cs/test/ReadAddressTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Threading.Tasks; @@ -82,14 +84,7 @@ internal Functions(bool preserveCopyUpdaterSource = false) } } - public override bool ConcurrentReader(ref KeyStruct key, ref ValueStruct input, ref ValueStruct value, ref Output output, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - output.value = SetReadOutput(key.key, value.value); - output.address = readInfo.Address; - return true; - } - - public override bool SingleReader(ref KeyStruct key, ref ValueStruct input, ref ValueStruct value, ref Output output, ref ReadInfo readInfo) + public override bool Reader(ref KeyStruct key, ref ValueStruct input, ref ValueStruct value, ref Output output, ref ReadInfo readInfo) { output.value = SetReadOutput(key.key, value.value); output.address = readInfo.Address; @@ -97,12 +92,12 @@ public override bool SingleReader(ref KeyStruct key, ref ValueStruct input, ref } // Return false to force a chain of values. - public override bool ConcurrentWriter(ref KeyStruct key, ref ValueStruct input, ref ValueStruct src, ref ValueStruct dst, ref Output output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => false; + public override bool InPlaceWriter(ref KeyStruct key, ref ValueStruct input, ref ValueStruct src, ref ValueStruct dst, ref Output output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) => false; public override bool InPlaceUpdater(ref KeyStruct key, ref ValueStruct input, ref ValueStruct value, ref Output output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) => false; // Record addresses - public override bool SingleWriter(ref KeyStruct key, ref ValueStruct input, ref ValueStruct src, ref ValueStruct dst, ref Output output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref KeyStruct key, ref ValueStruct input, ref ValueStruct src, ref ValueStruct dst, ref Output output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { dst = src; output.address = upsertInfo.Address; @@ -134,7 +129,7 @@ public override void ReadCompletionCallback(ref KeyStruct key, ref ValueStruct i if (useReadCache && readCopyOptions.CopyTo == ReadCopyTo.ReadCache) ClassicAssert.AreEqual(Constants.kInvalidAddress, recordMetadata.Address, $"key {key}"); else - ClassicAssert.AreEqual(output.address, recordMetadata.Address, $"key {key}"); // Should agree with what SingleWriter set + ClassicAssert.AreEqual(output.address, recordMetadata.Address, $"key {key}"); // Should agree with what InitialWriter set } } @@ -317,10 +312,7 @@ struct IterateKeyTestScanIteratorFunctions : IScanIteratorFunctions true; - public bool ConcurrentReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - - public bool SingleReader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref KeyStruct key, ref ValueStruct value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) { cursorRecordResult = CursorRecordResult.Accept; // default; not used here Output output = new() { address = recordMetadata.Address, value = SetReadOutput(key.key, value.value) }; @@ -578,4 +570,6 @@ public void ReadCopyOptionssMergeTest() } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs b/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs index adefeb6bbd6..e4f46005012 100644 --- a/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs +++ b/libs/storage/Tsavorite/cs/test/ReadCacheChainTests.cs @@ -10,7 +10,7 @@ using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; -using Tsavorite.test.LockableUnsafeContext; +using Tsavorite.test.TransactionalUnsafeContext; using Tsavorite.test.LockTable; using static Tsavorite.test.TestUtils; @@ -18,9 +18,9 @@ namespace Tsavorite.test.ReadCacheTests { - using LongAllocator = BlittableAllocator>>; - using LongStoreFunctions = StoreFunctions>; - using SpanByteStoreFunctions = StoreFunctions; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; internal static class RcTestGlobals { @@ -29,9 +29,9 @@ internal static class RcTestGlobals class ChainTests { - private TsavoriteKV store; + private TsavoriteKV store; private IDevice log; - private LongComparerModulo comparer; + private LongKeyComparerModulo comparer; const long LowChainKey = 40; const long MidChainKey = LowChainKey + ChainLen * (HashMod / 2); @@ -58,7 +58,7 @@ public void Setup() DeleteDirectory(MethodTestDir, wait: true); log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "NativeReadCacheTests.log"), deleteOnClose: true); - comparer = new LongComparerModulo(HashMod); + comparer = new LongKeyComparerModulo(HashMod); store = new(new() { IndexSize = 1L << 26, @@ -68,7 +68,7 @@ public void Setup() ReadCacheMemorySize = 1L << 15, ReadCachePageSize = 1L << 9, ReadCacheEnabled = true - }, StoreFunctions.Create(comparer) + }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -87,13 +87,16 @@ public enum RecordRegion { Immutable, OnDisk, Mutable }; void PopulateAndEvict(RecordRegion recordRegion = RecordRegion.OnDisk) { - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + if (recordRegion != RecordRegion.Immutable) { - for (int key = 0; key < NumKeys; key++) - _ = bContext.Upsert(key, key + ValueAdd); + for (int keyNum = 0; keyNum < NumKeys; keyNum++) + _ = bContext.Upsert(key.Set(keyNum), value.Set(keyNum + ValueAdd)); _ = bContext.CompletePending(true); if (recordRegion == RecordRegion.OnDisk) store.Log.FlushAndEvict(true); @@ -101,31 +104,35 @@ void PopulateAndEvict(RecordRegion recordRegion = RecordRegion.OnDisk) } // Two parts, so we can have some evicted (and bring them into the readcache), and some in immutable (readonly). - for (int key = 0; key < ImmutableSplitKey; key++) - _ = bContext.Upsert(key, key + ValueAdd); + for (int keyNum = 0; keyNum < ImmutableSplitKey; keyNum++) + _ = bContext.Upsert(key.Set(keyNum), value.Set(keyNum + ValueAdd)); _ = bContext.CompletePending(true); store.Log.FlushAndEvict(true); - for (long key = ImmutableSplitKey; key < NumKeys; key++) - _ = bContext.Upsert(key, key + ValueAdd); + for (long keyNum = ImmutableSplitKey; keyNum < NumKeys; keyNum++) + _ = bContext.Upsert(key.Set(keyNum), value.Set(keyNum + ValueAdd)); _ = bContext.CompletePending(true); store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, wait: true); } void CreateChain(RecordRegion recordRegion = RecordRegion.OnDisk) { - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; long output = -1; bool expectPending(long key) => recordRegion == RecordRegion.OnDisk || (recordRegion == RecordRegion.Immutable && key < ImmutableSplitKey); + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + // Pass1: PENDING reads and populate the cache for (long ii = 0; ii < ChainLen; ++ii) { - var key = LowChainKey + ii * HashMod; - var status = bContext.Read(key, out _); - if (expectPending(key)) + var keyNum = LowChainKey + ii * HashMod; + key.Set(keyNum); + var status = bContext.Read(key, ref output); + if (expectPending(keyNum)) { ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = bContext.CompletePendingWithOutputs(out var outputs, wait: true); @@ -133,24 +140,25 @@ void CreateChain(RecordRegion recordRegion = RecordRegion.OnDisk) ClassicAssert.IsTrue(status.Record.CopiedToReadCache, status.ToString()); } ClassicAssert.IsTrue(status.Found, status.ToString()); - if (key < MidChainKey) + if (keyNum < MidChainKey) readCacheBelowMidChainKeyEvictionAddress = store.ReadCache.TailAddress; } // Pass2: non-PENDING reads from the cache for (var ii = 0; ii < ChainLen; ++ii) { - var status = bContext.Read(LowChainKey + ii * HashMod, out _); + var status = bContext.Read(key.Set(LowChainKey + ii * HashMod), ref output); ClassicAssert.IsTrue(!status.IsPending && status.Found, status.ToString()); } // Pass 3: Put in bunch of extra keys into the cache so when we FlushAndEvict we get all the ones of interest. - for (var key = 0; key < NumKeys; ++key) + for (var keyNum = 0; keyNum < NumKeys; ++keyNum) { - if ((key % HashMod) != 0) + if ((keyNum % HashMod) != 0) { - var status = bContext.Read(key, out _); - if (expectPending(key)) + key.Set(keyNum); + var status = bContext.Read(key, ref output); + if (expectPending(keyNum)) { ClassicAssert.IsTrue(status.IsPending); _ = bContext.CompletePendingWithOutputs(out var outputs, wait: true); @@ -163,65 +171,65 @@ void CreateChain(RecordRegion recordRegion = RecordRegion.OnDisk) } } - unsafe bool GetRecordInInMemoryHashChain(long key, out bool isReadCache) + unsafe bool GetRecordInInMemoryHashChain(long keyNum, out bool isReadCache) { // returns whether the key was found before we'd go pending - var (la, pa) = GetHashChain(store, key, out long recordKey, out bool invalid, out isReadCache); + var (la, pa) = GetHashChain(store, SpanByte.FromPinnedVariable(ref keyNum), out var recordKey, out bool invalid, out isReadCache); while (isReadCache || la >= store.hlogBase.HeadAddress) { - if (recordKey == key && !invalid) + if (recordKey.ReadOnlySpan.AsRef() == keyNum && !invalid) return true; (la, pa) = NextInChain(store, pa, out recordKey, out invalid, ref isReadCache); } return false; } - internal bool FindRecordInReadCache(long key, out bool invalid, out long logicalAddress, out long physicalAddress) + internal bool FindRecordInReadCache(ReadOnlySpan key, out bool invalid, out long logicalAddress, out long physicalAddress) { // returns whether the key was found before we'd go pending - (logicalAddress, physicalAddress) = GetHashChain(store, key, out long recordKey, out invalid, out bool isReadCache); + (logicalAddress, physicalAddress) = GetHashChain(store, key, out var recordKey, out invalid, out bool isReadCache); while (isReadCache) { - if (recordKey == key) + if (recordKey.ReadOnlySpan.AsRef() == key.AsRef()) return true; (logicalAddress, physicalAddress) = NextInChain(store, physicalAddress, out recordKey, out invalid, ref isReadCache); } return false; } - internal static (long logicalAddress, long physicalAddress) GetHashChain(TsavoriteKV store, long key, out long recordKey, out bool invalid, out bool isReadCache) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static (long logicalAddress, long physicalAddress) GetHashChain(TsavoriteKV store, ReadOnlySpan key, out PinnedSpanByte recordKey, out bool invalid, out bool isReadCache) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - var tagExists = store.FindHashBucketEntryForKey(ref key, out var entry); + var tagExists = store.FindHashBucketEntryForKey(key, out var entry); ClassicAssert.IsTrue(tagExists); isReadCache = entry.ReadCache; var log = isReadCache ? store.readcache : store.hlog; var pa = log.GetPhysicalAddress(entry.Address & ~Constants.kReadCacheBitMask); - recordKey = log.GetKey(pa); - invalid = log.GetInfo(pa).Invalid; + recordKey = PinnedSpanByte.FromPinnedSpan(LogRecord.GetInlineKey(pa)); // Must return PinnedSpanByte to avoid scope issues with ReadOnlySpan + invalid = LogRecord.GetInfo(pa).Invalid; return (entry.Address, pa); } - (long logicalAddress, long physicalAddress) NextInChain(long physicalAddress, out long recordKey, out bool invalid, ref bool isReadCache) + (long logicalAddress, long physicalAddress) NextInChain(long physicalAddress, out PinnedSpanByte recordKey, out bool invalid, ref bool isReadCache) => NextInChain(store, physicalAddress, out recordKey, out invalid, ref isReadCache); - internal static (long logicalAddress, long physicalAddress) NextInChain(TsavoriteKV store, long physicalAddress, out long recordKey, out bool invalid, ref bool isReadCache) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static (long logicalAddress, long physicalAddress) NextInChain(TsavoriteKV store, long physicalAddress, out PinnedSpanByte recordKey, out bool invalid, ref bool isReadCache) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { var log = isReadCache ? store.readcache : store.hlog; - var info = log.GetInfo(physicalAddress); + var info = LogRecord.GetInfo(physicalAddress); var la = info.PreviousAddress; isReadCache = new HashBucketEntry { word = la }.ReadCache; log = isReadCache ? store.readcache : store.hlog; la &= ~Constants.kReadCacheBitMask; var pa = log.GetPhysicalAddress(la); - recordKey = log.GetKey(pa); - invalid = log.GetInfo(pa).Invalid; + recordKey = PinnedSpanByte.FromPinnedSpan(LogRecord.GetInlineKey(pa)); // Must return PinnedSpanByte to avoid scope issues with ReadOnlySpan + invalid = LogRecord.GetInfo(pa).Invalid; return (la, pa); } @@ -229,8 +237,11 @@ internal static (long logicalAddress, long physicalAddress) NextInChain= LowChainKey; expectedKey -= HashMod) + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + + var (la, pa) = GetHashChain(store, key.Set(LowChainKey), out var actualKey, out bool invalid, out bool isReadCache); + for (long expectedKey = HighChainKey; expectedKey >= LowChainKey; expectedKey -= HashMod) { // We evict from readcache only to just below midChainKey if (!evicted || expectedKey >= MidChainKey) @@ -238,13 +249,13 @@ internal static (long logicalAddress, long physicalAddress) NextInChain()); if (omitted.Contains(expectedKey)) ClassicAssert.IsTrue(invalid); } - else if (omitted.Contains(actualKey)) + else if (omitted.Contains(actualKey.ReadOnlySpan.AsRef())) { - ClassicAssert.AreEqual(deleted, store.hlog.GetInfo(pa).Tombstone); + ClassicAssert.AreEqual(deleted, LogRecord.GetInfo(pa).Tombstone); } (la, pa) = NextInChain(pa, out actualKey, out invalid, ref isReadCache); @@ -255,12 +266,12 @@ internal static (long logicalAddress, long physicalAddress) NextInChain key) => SkipReadCacheChain(store, key); - internal static (long logicalAddress, long physicalAddress) SkipReadCacheChain(TsavoriteKV store, long key) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static (long logicalAddress, long physicalAddress) SkipReadCacheChain(TsavoriteKV store, ReadOnlySpan key) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { var (la, pa) = GetHashChain(store, key, out _, out _, out bool isReadCache); while (isReadCache) @@ -268,15 +279,15 @@ internal static (long logicalAddress, long physicalAddress) SkipReadCacheChain expectedKey) { // Scan to the end of the readcache chain and verify we inserted the value. var (_, pa) = SkipReadCacheChain(expectedKey); - var storedKey = store.hlog.GetKey(pa); - ClassicAssert.AreEqual(expectedKey, storedKey); + var storedKey = LogRecord.GetInlineKey(pa); + ClassicAssert.AreEqual(expectedKey.AsRef(), storedKey.AsRef()); } - static void ClearCountsOnError(ClientSession, LongStoreFunctions, LongAllocator> luContext) + static void ClearCountsOnError(ClientSession luContext) { // If we already have an exception, clear these counts so "Run" will not report them spuriously. luContext.sharedLockCount = 0; @@ -304,15 +315,19 @@ public void DeleteCacheRecordTest() { PopulateAndEvict(); CreateChain(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - void doTest(long key) + void doTest(long keyNum) { + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + + key.Set(keyNum); var status = bContext.Delete(key); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); - status = bContext.Read(key, out var value); + status = bContext.Read(key, ref valueVal); ClassicAssert.IsFalse(status.Found, status.ToString()); } @@ -333,15 +348,19 @@ public void DeleteHalfOfAllReadCacheRecordsTest() { PopulateAndEvict(); CreateChain(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - void doTest(long key) + void doTest(long keyNum) { + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + + key.Set(keyNum); var status = bContext.Delete(key); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); - status = bContext.Read(key, out var value); + status = bContext.Read(key, ref valueVal); ClassicAssert.IsFalse(status.Found, status.ToString()); } @@ -400,18 +419,23 @@ void DoUpdateTest(bool useRMW) { PopulateAndEvict(); CreateChain(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - void doTest(long key) + void doTest(long keyNum) { - var status = bContext.Read(key, out var value); + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal), value = SpanByte.FromPinnedVariable(ref valueVal); + + key.Set(keyNum); + var status = bContext.Read(key, ref valueVal); ClassicAssert.IsTrue(status.Found, status.ToString()); + long input = valueVal + ValueAdd; if (useRMW) { // RMW will use the readcache entry for its source and then invalidate it. - status = bContext.RMW(key, value + ValueAdd); + status = bContext.RMW(key, ref input); ClassicAssert.IsTrue(status.Found && status.Record.CopyUpdated, status.ToString()); ClassicAssert.IsTrue(FindRecordInReadCache(key, out bool invalid, out _, out _)); @@ -419,13 +443,13 @@ void doTest(long key) } else { - status = bContext.Upsert(key, value + ValueAdd); + status = bContext.Upsert(key, value.Set(input)); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } - status = bContext.Read(key, out value); + status = bContext.Read(key, ref valueVal); ClassicAssert.IsTrue(status.Found, status.ToString()); - ClassicAssert.AreEqual(key + ValueAdd * 2, value); + ClassicAssert.AreEqual(keyNum + ValueAdd * 2, valueVal); } doTest(LowChainKey); @@ -446,13 +470,14 @@ public void SpliceInFromCTTTest() PopulateAndEvict(); CreateChain(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - long input = 0, output = 0, key = LowChainKey - HashMod; // key must be in evicted region for this test + long input = 0, output = 0, keyNum = LowChainKey - HashMod; // key must be in evicted region for this test + Span key = SpanByte.FromPinnedVariable(ref keyNum); ReadOptions readOptions = new() { CopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.MainLog) }; - var status = bContext.Read(ref key, ref input, ref output, ref readOptions, out _); + var status = bContext.Read(key, ref input, ref output, ref readOptions, out _); ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = bContext.CompletePending(wait: true); @@ -468,21 +493,22 @@ public void SpliceInFromUpsertTest([Values] RecordRegion recordRegion) PopulateAndEvict(recordRegion); CreateChain(recordRegion); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - long key = -1; + long keyNum = -1, valueNum = 0; + Span key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); if (recordRegion is RecordRegion.Immutable or RecordRegion.OnDisk) { - key = SpliceInExistingKey; - var status = bContext.Upsert(key, key + ValueAdd); + keyNum = SpliceInExistingKey; + var status = bContext.Upsert(key, value.Set(keyNum + ValueAdd)); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } else { - key = SpliceInNewKey; - var status = bContext.Upsert(key, key + ValueAdd); + keyNum = SpliceInNewKey; + var status = bContext.Upsert(key, value.Set(keyNum + ValueAdd)); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } @@ -498,15 +524,18 @@ public void SpliceInFromRMWTest([Values] RecordRegion recordRegion) PopulateAndEvict(recordRegion); CreateChain(recordRegion); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - long key = -1, output = -1; + long keyNum = -1, valueNum = 0, output = -1; + Span key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + + long input = keyNum + ValueAdd; if (recordRegion is RecordRegion.Immutable or RecordRegion.OnDisk) { // Existing key - key = SpliceInExistingKey; - var status = bContext.RMW(key, key + ValueAdd); + keyNum = SpliceInExistingKey; + var status = bContext.RMW(key, ref input); // If OnDisk, this used the readcache entry for its source and then invalidated it. ClassicAssert.IsTrue(status.Found && status.Record.CopyUpdated, status.ToString()); @@ -517,8 +546,8 @@ public void SpliceInFromRMWTest([Values] RecordRegion recordRegion) } { // New key - key = SpliceInNewKey; - status = bContext.RMW(key, key + ValueAdd); + keyNum = SpliceInNewKey; + status = bContext.RMW(key, ref input); // This NOTFOUND key will return PENDING because we have to trace back through the collisions. ClassicAssert.IsTrue(status.IsPending, status.ToString()); @@ -529,8 +558,8 @@ public void SpliceInFromRMWTest([Values] RecordRegion recordRegion) } else { - key = SpliceInNewKey; - var status = bContext.RMW(key, key + ValueAdd); + keyNum = SpliceInNewKey; + var status = bContext.RMW(key, ref input); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } @@ -546,19 +575,20 @@ public void SpliceInFromDeleteTest([Values] RecordRegion recordRegion) PopulateAndEvict(recordRegion); CreateChain(recordRegion); - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; - long key = -1; + long keyNum = -1; + Span key = SpanByte.FromPinnedVariable(ref keyNum); if (recordRegion is RecordRegion.Immutable or RecordRegion.OnDisk) { - key = SpliceInExistingKey; + keyNum = SpliceInExistingKey; var status = bContext.Delete(key); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } else { - key = SpliceInNewKey; + keyNum = SpliceInNewKey; var status = bContext.Delete(key); ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } @@ -575,18 +605,23 @@ public void VerifyLockCountsAfterReadCacheEvict() PopulateAndEvict(); CreateChain(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; + + var keyNums = GC.AllocateArray(3, pinned: true); + keyNums[0] = LowChainKey; + keyNums[1] = MidChainKey; + keyNums[2] = HighChainKey; var keys = new[] { - new FixedLengthLockableKeyStruct(LowChainKey, LockType.Exclusive, luContext), - new FixedLengthLockableKeyStruct(MidChainKey, LockType.Shared, luContext), - new FixedLengthLockableKeyStruct(HighChainKey, LockType.Exclusive, luContext) + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref keyNums[0]), LockType.Exclusive, luContext), + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref keyNums[1]), LockType.Shared, luContext), + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref keyNums[2]), LockType.Exclusive, luContext) }; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); try { @@ -598,7 +633,7 @@ public void VerifyLockCountsAfterReadCacheEvict() store.ReadCache.FlushAndEvict(wait: true); int xlocks = 0, slocks = 0; - foreach (var idx in LockableUnsafeContextTests.EnumActionKeyIndices(keys, LockableUnsafeContextTests.LockOperationType.Unlock)) + foreach (var idx in TransactionalUnsafeContextTests.EnumActionKeyIndices(keys, TransactionalUnsafeContextTests.LockOperationType.Unlock)) { if (keys[idx].LockType == LockType.Exclusive) ++xlocks; @@ -607,10 +642,10 @@ public void VerifyLockCountsAfterReadCacheEvict() } AssertTotalLockCounts(xlocks, slocks); - foreach (var idx in LockableUnsafeContextTests.EnumActionKeyIndices(keys, LockableUnsafeContextTests.LockOperationType.Unlock)) + foreach (var idx in TransactionalUnsafeContextTests.EnumActionKeyIndices(keys, TransactionalUnsafeContextTests.LockOperationType.Unlock)) { ref var key = ref keys[idx]; - HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(ref key.Key)); + HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(key.Key.ReadOnlySpan)); OverflowBucketLockTableTests.PopulateHei(store, ref hei); var lockState = store.LockTable.GetLockState(ref hei); @@ -632,7 +667,7 @@ public void VerifyLockCountsAfterReadCacheEvict() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } @@ -642,9 +677,9 @@ public void VerifyLockCountsAfterReadCacheEvict() class LongStressChainTests { - private TsavoriteKV store; + private TsavoriteKV store; private IDevice log; - private LongComparerModulo comparer; + private LongKeyComparerModulo comparer; const long ValueAdd = 1_000_000_000; const long NumKeys = 2_000; @@ -675,7 +710,7 @@ public void Setup() } } - comparer = new LongComparerModulo((long)modRange); + comparer = new LongKeyComparerModulo((long)modRange); // Make the main log small enough that we force the readcache store = new(new() @@ -687,7 +722,7 @@ public void Setup() ReadCacheMemorySize = 1L << 15, ReadCachePageSize = 1L << 9, ReadCacheEnabled = true - }, StoreFunctions.Create(comparer) + }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -702,38 +737,10 @@ public void TearDown() DeleteDirectory(MethodTestDir); } - internal class RmwLongFunctions : SimpleSessionFunctions + internal class RmwLongFunctions : SimpleLongSimpleFunctions { /// - public override bool ConcurrentWriter(ref long key, ref long input, ref long src, ref long dst, ref long output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) - { - dst = output = src; - return true; - } - - /// - public override bool SingleWriter(ref long key, ref long input, ref long src, ref long dst, ref long output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - dst = output = src; - return true; - } - - /// - public override bool CopyUpdater(ref long key, ref long input, ref long oldValue, ref long newValue, ref long output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - newValue = output = input; - return true; - } - - /// - public override bool InPlaceUpdater(ref long key, ref long input, ref long value, ref long output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - value = output = input; - return true; - } - - /// - public override bool InitialUpdater(ref long key, ref long input, ref long value, ref long output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { Assert.Fail("For these tests, InitialUpdater should never be called"); return false; @@ -742,13 +749,13 @@ public override bool InitialUpdater(ref long key, ref long input, ref long value unsafe void PopulateAndEvict() { - using var session = store.NewSession>(new SimpleSessionFunctions()); + using var session = store.NewSession(new RmwLongFunctions()); var bContext = session.BasicContext; for (long ii = 0; ii < NumKeys; ii++) { long key = ii; - var status = bContext.Upsert(ref key, ref key); + var status = bContext.Upsert(SpanByte.FromPinnedVariable(ref key), SpanByte.FromPinnedVariable(ref key)); ClassicAssert.IsFalse(status.IsPending); ClassicAssert.IsTrue(status.Record.Created, $"key {key}, status {status}"); } @@ -778,7 +785,7 @@ public void LongRcMultiThreadTest([Values] HashModulo modRange, [Values(0, 1, 2, const int numIterations = 1; unsafe void runReadThread(int tid) { - using var session = store.NewSession>(new SimpleSessionFunctions()); + using var session = store.NewSession(new RmwLongFunctions()); var bContext = session.BasicContext; for (var iteration = 0; iteration < numIterations; ++iteration) @@ -787,7 +794,7 @@ unsafe void runReadThread(int tid) for (var ii = 0; ii < NumKeys; ++ii) { long key = ii, output = 0; - var status = bContext.Read(ref key, ref output); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key), ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -810,7 +817,7 @@ unsafe void runReadThread(int tid) status = completedOutputs.Current.Status; output = completedOutputs.Current.Output; - key = completedOutputs.Current.Key; + key = completedOutputs.Current.Key.AsRef(); ClassicAssert.AreEqual(completedOutputs.Current.RecordMetadata.Address == Constants.kInvalidAddress, status.Record.CopiedToReadCache, $"key {key}: {status}"); ClassicAssert.IsTrue(status.Found, $"key {key}, status {status}, wasPending {true}"); ClassicAssert.AreEqual(key, output % ValueAdd); @@ -834,8 +841,8 @@ unsafe void runUpdateThread(int tid) { long key = ii, input = ii + ValueAdd * tid, output = 0; var status = updateOp == UpdateOp.RMW - ? bContext.RMW(ref key, ref input, ref output) - : bContext.Upsert(ref key, ref input, ref input, ref output); + ? bContext.RMW(SpanByte.FromPinnedVariable(ref key), ref input, ref output) + : bContext.Upsert(SpanByte.FromPinnedVariable(ref key), ref input, SpanByte.FromPinnedVariable(ref input), ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -860,8 +867,8 @@ unsafe void runUpdateThread(int tid) { ++numCompleted; if (updateOp == UpdateOp.RMW) // Upsert will not try to find records below HeadAddress, but it may find them in-memory - ClassicAssert.IsTrue(completedOutputs.Current.Status.Found, $"key {completedOutputs.Current.Key}, status {completedOutputs.Current.Status}, wasPending {true}"); - ClassicAssert.AreEqual(completedOutputs.Current.Key + ValueAdd * tid, completedOutputs.Current.Output); + ClassicAssert.IsTrue(completedOutputs.Current.Status.Found, $"key {completedOutputs.Current.Key.ToShortString()}, status {completedOutputs.Current.Status}, wasPending {true}"); + ClassicAssert.AreEqual(completedOutputs.Current.Key.AsRef() + ValueAdd * tid, completedOutputs.Current.Output); } } } @@ -886,9 +893,9 @@ unsafe void runUpdateThread(int tid) class SpanByteStressChainTests { - private TsavoriteKV> store; + private TsavoriteKV> store; private IDevice log; - SpanByteComparerModulo comparer; + SpanByteKeyComparerModulo comparer; const long ValueAdd = 1_000_000_000; @@ -920,7 +927,7 @@ public void Setup() } } - comparer = new SpanByteComparerModulo(modRange); + comparer = new SpanByteKeyComparerModulo(modRange); // Make the main log small enough that we force the readcache store = new(new() @@ -932,7 +939,7 @@ public void Setup() ReadCacheMemorySize = 1L << 15, ReadCachePageSize = 1L << 9, ReadCacheEnabled = true - }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) + }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -950,40 +957,44 @@ public void TearDown() internal class RmwSpanByteFunctions : SpanByteFunctions { /// - public override bool ConcurrentWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - src.CopyTo(ref dst); - src.CopyTo(ref output, memoryPool); + if (!base.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo)) + return false; + srcValue.CopyTo(ref output, memoryPool); return true; } /// - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - src.CopyTo(ref dst); - src.CopyTo(ref output, memoryPool); + if (!base.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo)) + return false; + srcValue.CopyTo(ref output, memoryPool); return true; } /// - public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - input.CopyTo(ref newValue); + if (!base.CopyUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) + return false; input.CopyTo(ref output, memoryPool); return true; } /// - public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { // The default implementation of IPU simply writes input to destination, if there is space - base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + if (!base.InPlaceUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo)) + return false; input.CopyTo(ref output, memoryPool); return true; } /// - public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { Assert.Fail("For these tests, InitialUpdater should never be called"); return false; @@ -992,16 +1003,15 @@ public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref Sp unsafe void PopulateAndEvict() { - using var session = store.NewSession>(new SpanByteFunctions()); + using var session = store.NewSession>(new SpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[sizeof(long)]; for (long ii = 0; ii < NumKeys; ii++) { - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - var status = bContext.Upsert(ref key, ref key); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + var status = bContext.Upsert(key, key); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } bContext.CompletePending(true); @@ -1028,11 +1038,10 @@ public void SpanByteRcMultiThreadTest([Values] HashModulo modRange, [Values(0, 1 const int numIterations = 1; unsafe void runReadThread(int tid) { - using var session = store.NewSession>(new SpanByteFunctions()); + using var session = store.NewSession>(new SpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[sizeof(long)]; for (var iteration = 0; iteration < numIterations; ++iteration) { @@ -1041,8 +1050,8 @@ unsafe void runReadThread(int tid) { SpanByteAndMemory output = default; - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - var status = bContext.Read(ref key, ref output); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + var status = bContext.Read(key, ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -1053,7 +1062,7 @@ unsafe void runReadThread(int tid) ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {ii}, {status}, wasPending {false}, pt 1"); ClassicAssert.IsNotNull(output.Memory, $"tid {tid}, key {ii}, wasPending {false}, pt 2"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.ReadOnlySpan); ClassicAssert.AreEqual(ii, value % ValueAdd, $"tid {tid}, key {ii}, wasPending {false}, pt 3"); output.Memory.Dispose(); } @@ -1070,13 +1079,13 @@ unsafe void runReadThread(int tid) status = completedOutputs.Current.Status; output = completedOutputs.Current.Output; // Note: do NOT overwrite 'key' here - long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key.AsReadOnlySpan()); + long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key); ClassicAssert.AreEqual(completedOutputs.Current.RecordMetadata.Address == Constants.kInvalidAddress, status.Record.CopiedToReadCache, $"key {keyLong}: {status}"); ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {keyLong}, {status}, wasPending {true}, pt 1"); ClassicAssert.IsNotNull(output.Memory, $"tid {tid}, key {keyLong}, wasPending {true}, pt 2"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.ReadOnlySpan); ClassicAssert.AreEqual(keyLong, value % ValueAdd, $"tid {tid}, key {keyLong}, wasPending {true}, pt 3"); output.Memory.Dispose(); } @@ -1089,13 +1098,12 @@ unsafe void runReadThread(int tid) unsafe void runUpdateThread(int tid) { - using var session = store.NewSession>(new RmwSpanByteFunctions()); + using var session = store.NewSession>(new RmwSpanByteFunctions()); var bContext = session.BasicContext; - Span keyVec = stackalloc byte[sizeof(long)]; - var key = SpanByte.FromPinnedSpan(keyVec); - Span inputVec = stackalloc byte[sizeof(long)]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[sizeof(long)]; + Span input = stackalloc byte[sizeof(long)]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); for (var iteration = 0; iteration < numIterations; ++iteration) { @@ -1104,11 +1112,11 @@ unsafe void runUpdateThread(int tid) { SpanByteAndMemory output = default; - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(keyVec, ii)); - ClassicAssert.IsTrue(BitConverter.TryWriteBytes(inputVec, ii + ValueAdd)); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(key, ii)); + ClassicAssert.IsTrue(BitConverter.TryWriteBytes(input, ii + ValueAdd)); var status = updateOp == UpdateOp.RMW - ? bContext.RMW(ref key, ref input, ref output) - : bContext.Upsert(ref key, ref input, ref input, ref output); + ? bContext.RMW(key, ref pinnedInputSpan, ref output) + : bContext.Upsert(key, ref pinnedInputSpan, input, ref output); var numPending = ii - numCompleted; if (status.IsPending) @@ -1122,7 +1130,7 @@ unsafe void runUpdateThread(int tid) if (updateOp == UpdateOp.RMW) // Upsert will not try to find records below HeadAddress, but it may find them in-memory ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {ii}, {status}"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.ReadOnlySpan); ClassicAssert.AreEqual(ii + ValueAdd, value, $"tid {tid}, key {ii}, wasPending {false}"); output.Memory?.Dispose(); @@ -1140,12 +1148,12 @@ unsafe void runUpdateThread(int tid) status = completedOutputs.Current.Status; output = completedOutputs.Current.Output; // Note: do NOT overwrite 'key' here - long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key.AsReadOnlySpan()); + long keyLong = BitConverter.ToInt64(completedOutputs.Current.Key); if (updateOp == UpdateOp.RMW) // Upsert will not try to find records below HeadAddress, but it may find them in-memory ClassicAssert.IsTrue(status.Found, $"tid {tid}, key {keyLong}, {status}"); - long value = BitConverter.ToInt64(output.AsReadOnlySpan()); + long value = BitConverter.ToInt64(output.ReadOnlySpan); ClassicAssert.AreEqual(keyLong + ValueAdd, value, $"tid {tid}, key {keyLong}, wasPending {true}"); output.Memory?.Dispose(); diff --git a/libs/storage/Tsavorite/cs/test/RecoverReadOnlyTest.cs b/libs/storage/Tsavorite/cs/test/RecoverReadOnlyTest.cs index 7c38c13cc6c..dbf4466cc9a 100644 --- a/libs/storage/Tsavorite/cs/test/RecoverReadOnlyTest.cs +++ b/libs/storage/Tsavorite/cs/test/RecoverReadOnlyTest.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Text; @@ -127,4 +129,6 @@ static async Task BeginRecoverReadOnlyLoop(TsavoriteLog log, CancellationToken c } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs index a13d60f9d95..335e35a6f8f 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Threading.Tasks; @@ -46,7 +48,7 @@ protected void BaseTearDown() TestUtils.DeleteDirectory(TestUtils.MethodTestDir); } - public class MyFunctions : SimpleSimpleFunctions + public class MyFunctions : SimpleLongSimpleFunctions { public override void ReadCompletionCallback(ref long key, ref long input, ref long output, Empty ctx, Status status, RecordMetadata recordMetadata) { @@ -55,7 +57,7 @@ public override void ReadCompletionCallback(ref long key, ref long input, ref lo } } - public class MyFunctions2 : SimpleSimpleFunctions + public class MyFunctions2 : SimpleLongSimpleFunctions { public override void ReadCompletionCallback(ref long key, ref long input, ref long output, Empty ctx, Status status, RecordMetadata recordMetadata) { @@ -203,7 +205,7 @@ public async ValueTask RecoveryCheck2( , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s1 = store1.NewSession>(new SimpleSimpleFunctions()); + using var s1 = store1.NewSession>(new SimpleLongSimpleFunctions()); var bc1 = s1.BasicContext; using var store2 = new TsavoriteKV(new() @@ -259,7 +261,7 @@ public async ValueTask RecoveryCheck2( ClassicAssert.AreEqual(store1.Log.ReadOnlyAddress, store2.Log.ReadOnlyAddress); ClassicAssert.AreEqual(store1.Log.TailAddress, store2.Log.TailAddress); - using var s2 = store2.NewSession>(new SimpleSimpleFunctions()); + using var s2 = store2.NewSession>(new SimpleLongSimpleFunctions()); var bc2 = s2.BasicContext; for (long key = 0; key < 1000 * i + 1000; key++) { @@ -300,7 +302,7 @@ public void RecoveryCheck2Repeated( if (i > 0) _ = store.Recover(default, token); - using var s1 = store.NewSession>(new SimpleSimpleFunctions()); + using var s1 = store.NewSession>(new SimpleLongSimpleFunctions()); var bc1 = s1.BasicContext; for (long key = 1000 * i; key < 1000 * i + 1000; key++) @@ -313,7 +315,7 @@ public void RecoveryCheck2Repeated( (success, token) = task.AsTask().GetAwaiter().GetResult(); ClassicAssert.IsTrue(success); - using var s2 = store.NewSession>(new SimpleSimpleFunctions()); + using var s2 = store.NewSession>(new SimpleLongSimpleFunctions()); var bc2 = s2.BasicContext; for (long key = 0; key < 1000 * i + 1000; key++) @@ -349,7 +351,7 @@ public void RecoveryRollback( , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s1 = store.NewSession>(new SimpleSimpleFunctions()); + using var s1 = store.NewSession>(new SimpleLongSimpleFunctions()); var bc1 = s1.BasicContext; for (long key = 0; key < 1000; key++) @@ -503,7 +505,7 @@ public async ValueTask RecoveryCheck3( , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s1 = store1.NewSession>(new SimpleSimpleFunctions()); + using var s1 = store1.NewSession>(new SimpleLongSimpleFunctions()); var bc1 = s1.BasicContext; using var store2 = new TsavoriteKV(new() @@ -559,7 +561,7 @@ public async ValueTask RecoveryCheck3( ClassicAssert.AreEqual(store1.Log.ReadOnlyAddress, store2.Log.ReadOnlyAddress); ClassicAssert.AreEqual(store1.Log.TailAddress, store2.Log.TailAddress); - using var s2 = store2.NewSession>(new SimpleSimpleFunctions()); + using var s2 = store2.NewSession>(new SimpleLongSimpleFunctions()); var bc2 = s2.BasicContext; for (long key = 0; key < 1000 * i + 1000; key++) { @@ -605,7 +607,7 @@ public async ValueTask RecoveryCheck4( , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - using var s1 = store1.NewSession>(new SimpleSimpleFunctions()); + using var s1 = store1.NewSession>(new SimpleLongSimpleFunctions()); var bc1 = s1.BasicContext; using var store2 = new TsavoriteKV(new() @@ -664,7 +666,7 @@ public async ValueTask RecoveryCheck4( ClassicAssert.AreEqual(store1.Log.ReadOnlyAddress, store2.Log.ReadOnlyAddress); ClassicAssert.AreEqual(store1.Log.TailAddress, store2.Log.TailAddress); - using var s2 = store2.NewSession>(new SimpleSimpleFunctions()); + using var s2 = store2.NewSession>(new SimpleLongSimpleFunctions()); var bc2 = s2.BasicContext; for (long key = 0; key < 1000 * i + 1000; key++) { @@ -1083,4 +1085,5 @@ public async ValueTask StreamingSnapshotBasicTest([Values] bool isAsync, [Values _ = bc2.CompletePending(true); } } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/RecoveryTestTypes.cs b/libs/storage/Tsavorite/cs/test/RecoveryTestTypes.cs index f5f9da583bd..4d4c3b8d00a 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryTestTypes.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryTestTypes.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.Threading; using Tsavorite.core; @@ -45,13 +47,7 @@ public struct Output public class Functions : SessionFunctionsBase { // Read functions - public override bool SingleReader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo) - { - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo) { dst.value = value; return true; @@ -78,4 +74,6 @@ public override bool CopyUpdater(ref AdId key, ref AdInput input, ref NumClicks return true; } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/RecoveryTests.cs b/libs/storage/Tsavorite/cs/test/RecoveryTests.cs index 6951b439d58..752e8cbabfa 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryTests.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -358,7 +360,7 @@ private async ValueTask RunTest(AllocatorTyp private void Populate(TsavoriteKV store) { - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession>(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; for (int i = 0; i < DeviceTypeRecoveryTests.NumOps; i++) @@ -368,7 +370,7 @@ private void Populate(TsavoriteKV static int GetRandomLength(Random r) => r.Next(StackAllocMax) + 1; // +1 to remain in range 1..StackAllocMax - private unsafe void Populate(TsavoriteKV> store) + private unsafe void Populate(TsavoriteKV> store) { using var session = store.NewSession(new VLVectorFunctions()); var bContext = session.BasicContext; @@ -439,7 +441,7 @@ private async ValueTask RecoverAndReadTest(TsavoriteKV store) { - using var session = store.NewSession>(new SimpleSimpleFunctions()); + using var session = store.NewSession>(new SimpleLongSimpleFunctions()); var bContext = session.BasicContext; for (var i = 0; i < DeviceTypeRecoveryTests.NumUniqueKeys; i++) @@ -450,13 +452,13 @@ private static void Read(TsavoriteKV> store, bool isAsync) + private async ValueTask RecoverAndReadTest(TsavoriteKV> store, bool isAsync) { await Recover(store, isAsync); Read(store); } - private static void Read(TsavoriteKV> store) + private static void Read(TsavoriteKV> store) { using var session = store.NewSession(new VLVectorFunctions()); var bContext = session.BasicContext; @@ -511,4 +513,6 @@ private async ValueTask Recover(TsavoriteKV< _ = store.Recover(indexToken, logToken); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/ReproReadCacheTest.cs b/libs/storage/Tsavorite/cs/test/ReproReadCacheTest.cs index 280f6b3938f..546f6ab29d4 100644 --- a/libs/storage/Tsavorite/cs/test/ReproReadCacheTest.cs +++ b/libs/storage/Tsavorite/cs/test/ReproReadCacheTest.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Buffers; using System.Collections.Generic; @@ -15,17 +17,14 @@ namespace Tsavorite.test.ReadCacheTests { - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; [TestFixture] internal class RandomReadCacheTests { class Functions : SpanByteFunctions { - public override bool ConcurrentReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - => SingleReader(ref key, ref input, ref value, ref dst, ref readInfo); - - public override bool SingleReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) + public override bool Reader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory dst, ref ReadInfo readInfo) { var keyString = new string(MemoryMarshal.Cast(key.AsReadOnlySpan())); var inputString = new string(MemoryMarshal.Cast(input.AsReadOnlySpan())); @@ -52,7 +51,7 @@ public override void ReadCompletionCallback(ref SpanByte key, ref SpanByte input } IDevice log = default; - TsavoriteKV> store = default; + TsavoriteKV> store = default; [SetUp] public void Setup() @@ -61,7 +60,7 @@ public void Setup() string filename = Path.Join(MethodTestDir, "BasicTests.log"); - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = 1L << 26, MemorySize = 1L << 15, @@ -230,4 +229,6 @@ void LocalRun(int startKey, int endKey) Task.WaitAll([.. tasks]); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/RevivificationTests.cs b/libs/storage/Tsavorite/cs/test/RevivificationTests.cs index ac965b26ca1..53f16047fd3 100644 --- a/libs/storage/Tsavorite/cs/test/RevivificationTests.cs +++ b/libs/storage/Tsavorite/cs/test/RevivificationTests.cs @@ -17,7 +17,7 @@ namespace Tsavorite.test.Revivification { // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - internal readonly struct RevivificationSpanByteComparer : IKeyComparer + internal readonly struct RevivificationSpanByteComparer : IKeyComparer { private readonly SpanByteComparer defaultComparer; private readonly int collisionRange; @@ -28,22 +28,24 @@ internal RevivificationSpanByteComparer(CollisionRange range) collisionRange = (int)range; } - public bool Equals(ref SpanByte k1, ref SpanByte k2) => defaultComparer.Equals(ref k1, ref k2); + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => defaultComparer.Equals(k1, k2); // The hash code ends with 0 so mod Ten isn't so helpful, so shift - public long GetHashCode64(ref SpanByte k) => (defaultComparer.GetHashCode64(ref k) >> 4) % collisionRange; + public long GetHashCode64(ReadOnlySpan k) => (defaultComparer.GetHashCode64(k) >> 4) % collisionRange; } } namespace Tsavorite.test.Revivification { +#if LOGRECORD_TODO using ClassAllocator = GenericAllocator>>; using ClassStoreFunctions = StoreFunctions>; +#endif // LOGRECORD_TODO - using IntAllocator = BlittableAllocator>>; - using IntStoreFunctions = StoreFunctions>; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; - using SpanByteStoreFunctions = StoreFunctions; + using SpanByteStoreFunctions = StoreFunctions; public enum DeleteDest { FreeList, InChain } @@ -57,6 +59,18 @@ public enum RecordElision { Elide, NoElide } struct RevivificationTestUtils { + internal static RevivificationSettings FixedLengthBins = new() + { + FreeRecordBins = + [ + 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 + BestFitScanLimit = RevivificationBin.UseFirstFit + } + ] + }; + internal const double HalfOfMutableFraction = 0.5; // Half of the mutable region internal static double GetRevivifiableFraction(RevivifiableFraction frac) @@ -74,30 +88,28 @@ internal static RMWInfo CopyToRMWInfo(ref UpsertInfo upsertInfo) SessionID = upsertInfo.SessionID, Address = upsertInfo.Address, KeyHash = upsertInfo.KeyHash, - UsedValueLength = upsertInfo.UsedValueLength, - FullValueLength = upsertInfo.FullValueLength, Action = RMWAction.Default, }; - internal static FreeRecordPool CreateSingleBinFreeRecordPool( - TsavoriteKV store, RevivificationBin binDef, int fixedRecordLength = 0) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - => new(store, new RevivificationSettings() { FreeRecordBins = [binDef] }, fixedRecordLength); + internal static FreeRecordPool CreateSingleBinFreeRecordPool( + TsavoriteKV store, RevivificationBin binDef) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + => new(store, new RevivificationSettings() { FreeRecordBins = [binDef] }); - internal static bool HasRecords(TsavoriteKV store) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static bool HasRecords(TsavoriteKV store) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => HasRecords(store.RevivificationManager.FreeRecordPool); - internal static bool HasRecords(TsavoriteKV store, FreeRecordPool pool) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static bool HasRecords(TsavoriteKV store, FreeRecordPool pool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => HasRecords(pool ?? store.RevivificationManager.FreeRecordPool); - internal static bool HasRecords(FreeRecordPool pool) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static bool HasRecords(FreeRecordPool pool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { if (pool is not null) { @@ -110,10 +122,10 @@ internal static bool HasRecords(FreeR return false; } - internal static FreeRecordPool SwapFreeRecordPool( - TsavoriteKV store, FreeRecordPool inPool) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static FreeRecordPool SwapFreeRecordPool( + TsavoriteKV store, FreeRecordPool inPool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { var pool = store.RevivificationManager.FreeRecordPool; store.RevivificationManager.FreeRecordPool = inPool; @@ -122,45 +134,45 @@ internal static FreeRecordPool SwapFr internal const int DefaultRecordWaitTimeoutMs = 2000; - internal static bool GetBinIndex(FreeRecordPool pool, int recordSize, out int binIndex) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static bool GetBinIndex(FreeRecordPool pool, int recordSize, out int binIndex) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.GetBinIndex(recordSize, out binIndex); - internal static int GetBinCount(FreeRecordPool pool) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static int GetBinCount(FreeRecordPool pool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.bins.Length; - internal static int GetRecordCount(FreeRecordPool pool, int binIndex) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static int GetRecordCount(FreeRecordPool pool, int binIndex) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.bins[binIndex].recordCount; - internal static int GetMaxRecordSize(FreeRecordPool pool, int binIndex) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static int GetMaxRecordSize(FreeRecordPool pool, int binIndex) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.bins[binIndex].maxRecordSize; - internal static unsafe bool IsSet(FreeRecordPool pool, int binIndex, int recordIndex) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe bool IsSet(FreeRecordPool pool, int binIndex, int recordIndex) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.bins[binIndex].records[recordIndex].IsSet; - internal static bool TryTakeFromBin(FreeRecordPool pool, int binIndex, int recordSize, long minAddress, - TsavoriteKV store, out long address, ref RevivificationStats revivStats) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - => pool.bins[binIndex].TryTake(recordSize, minAddress, store, out address, ref revivStats); + internal static bool TryTakeFromBin(FreeRecordPool pool, int binIndex, ref RecordSizeInfo sizeInfo, long minAddress, + TsavoriteKV store, out long address, ref RevivificationStats revivStats) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + => pool.bins[binIndex].TryTake(ref sizeInfo, minAddress, store, out address, ref revivStats); - internal static int GetSegmentStart(FreeRecordPool pool, int binIndex, int recordSize) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static int GetSegmentStart(FreeRecordPool pool, int binIndex, int recordSize) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => pool.bins[binIndex].GetSegmentStart(recordSize); - internal static void WaitForRecords(TsavoriteKV store, bool want, FreeRecordPool pool = default) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static void WaitForRecords(TsavoriteKV store, bool want, FreeRecordPool pool = default) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { pool ??= store.RevivificationManager.FreeRecordPool; @@ -179,14 +191,14 @@ internal static void WaitForRecords(T } } - internal static unsafe int GetFreeRecordCount(TsavoriteKV store) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe int GetFreeRecordCount(TsavoriteKV store) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => GetFreeRecordCount(store.RevivificationManager.FreeRecordPool); - internal static unsafe int GetFreeRecordCount(FreeRecordPool pool) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe int GetFreeRecordCount(FreeRecordPool pool) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { // This returns the count of all records, not just the free ones. var count = 0; @@ -204,36 +216,31 @@ internal static unsafe int GetFreeRecordCount(TsavoriteKV store, TKey key) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - => AssertElidable(store, ref key); - - internal static void AssertElidable(TsavoriteKV store, ref TKey key) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static void AssertElidable(TsavoriteKV store, ReadOnlySpan key) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - OperationStackContext stackCtx = new(store.storeFunctions.GetKeyHashCode64(ref key)); - ClassicAssert.IsTrue(store.FindTag(ref stackCtx.hei), $"AssertElidable: Cannot find key {key}"); - var recordInfo = store.hlog.GetInfo(store.hlog.GetPhysicalAddress(stackCtx.hei.Address)); + OperationStackContext 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)); 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 + internal static int GetRevivifiableRecordCount(TsavoriteKV store, int numRecords) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => (int)(numRecords * store.RevivificationManager.revivifiableFraction); - internal static int GetMinRevivifiableKey(TsavoriteKV store, int numRecords) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static int GetMinRevivifiableKey(TsavoriteKV store, int numRecords) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => numRecords - GetRevivifiableRecordCount(store, numRecords); } [TestFixture] class RevivificationFixedLenTests { - internal class RevivificationFixedLenFunctions : SimpleSimpleFunctions + internal class RevivificationFixedLenFunctions : SimpleLongSimpleFunctions { } @@ -242,9 +249,9 @@ internal class RevivificationFixedLenFunctions : SimpleSimpleFunctions RevivificationFixedLenFunctions functions; - private TsavoriteKV store; - private ClientSession session; - private BasicContext bContext; + private TsavoriteKV store; + private ClientSession session; + private BasicContext bContext; private IDevice log; [SetUp] @@ -269,7 +276,7 @@ public void Setup() } } - var revivificationSettings = RevivificationSettings.DefaultFixedLength.Clone(); + var revivificationSettings = RevivificationTestUtils.FixedLengthBins.Clone(); if (revivifiableFraction.HasValue) revivificationSettings.RevivifiableFraction = revivifiableFraction.Value; if (recordElision.HasValue) @@ -281,10 +288,10 @@ public void Setup() PageSize = 1L << 12, MemorySize = 1L << 20, RevivificationSettings = revivificationSettings - }, StoreFunctions.Create(IntKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); functions = new RevivificationFixedLenFunctions(); - session = store.NewSession(functions); + session = store.NewSession(functions); bContext = session.BasicContext; } @@ -303,9 +310,10 @@ public void TearDown() void Populate() { - for (int key = 0; key < NumRecords; key++) + for (long keyNum = 0; keyNum < NumRecords; keyNum++) { - var status = bContext.Upsert(key, key * ValueMult); + long valueNum = keyNum * ValueMult; + var status = bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref valueNum)); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } } @@ -321,7 +329,8 @@ public void SimpleFixedLenTest([Values] DeleteDest deleteDest, [Values(UpdateOp. if (stayInChain) _ = RevivificationTestUtils.SwapFreeRecordPool(store, default); - var deleteKey = RevivificationTestUtils.GetMinRevivifiableKey(store, NumRecords); + long deleteKeyNum = RevivificationTestUtils.GetMinRevivifiableKey(store, NumRecords); + var deleteKey = SpanByte.FromPinnedVariable(ref deleteKeyNum); if (!stayInChain) RevivificationTestUtils.AssertElidable(store, deleteKey); var tailAddress = store.Log.TailAddress; @@ -329,8 +338,9 @@ public void SimpleFixedLenTest([Values] DeleteDest deleteDest, [Values(UpdateOp. _ = bContext.Delete(deleteKey); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); - var updateKey = deleteDest == DeleteDest.InChain ? deleteKey : NumRecords + 1; - var updateValue = updateKey + ValueMult; + long updateKeyNum = deleteDest == DeleteDest.InChain ? deleteKeyNum : NumRecords + 1; + var updateValueNum = updateKeyNum + ValueMult; + Span updateKey = SpanByte.FromPinnedVariable(ref updateKeyNum), updateValue = SpanByte.FromPinnedVariable(ref updateValueNum); if (!stayInChain) { @@ -338,7 +348,7 @@ public void SimpleFixedLenTest([Values] DeleteDest deleteDest, [Values(UpdateOp. RevivificationTestUtils.WaitForRecords(store, want: true); } - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(updateKey, updateValue) : bContext.RMW(updateKey, updateValue); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(updateKey, updateValue) : bContext.RMW(updateKey, ref updateValueNum); if (!stayInChain) RevivificationTestUtils.WaitForRecords(store, want: false); @@ -355,26 +365,28 @@ public void UnelideTest([Values] RecordElision elision, [Values(UpdateOp.Upsert, var tailAddress = store.Log.TailAddress; // First delete all keys. This will overflow the bin. - for (var key = 0; key < NumRecords; ++key) + for (long keyNum = 0; keyNum < NumRecords; ++keyNum) { - _ = bContext.Delete(key); + _ = bContext.Delete(SpanByte.FromPinnedVariable(ref keyNum)); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); } - ClassicAssert.AreEqual(RevivificationBin.DefaultRecordsPerBin, RevivificationTestUtils.GetFreeRecordCount(store)); + // The NumberOfRecords will be adjusted upward so the partition is cache-line aligned, so this may be higher than specified. + ClassicAssert.LessOrEqual(RevivificationBin.DefaultRecordsPerBin, RevivificationTestUtils.GetFreeRecordCount(store)); RevivificationTestUtils.WaitForRecords(store, want: true); // Now re-add the keys. - for (var key = 0; key < NumRecords; ++key) + for (long keyNum = 0; keyNum < NumRecords; ++keyNum) { - var value = key + ValueMult; - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, value) : bContext.RMW(key, value); + long valueNum = keyNum + ValueMult; + Span key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref valueNum); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, value) : bContext.RMW(key, ref valueNum); } // 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 = RecordInfo.GetLength() + sizeof(int) * 2; + var recordSize = RoundUp(RecordInfo.GetLength() + (sizeof(int) + sizeof(long)) * 2, Constants.kRecordAlignment); var numIneligibleRecords = NumRecords - RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords); var noElisionExpectedTailAddress = tailAddress + numIneligibleRecords * recordSize; @@ -394,11 +406,13 @@ public void SimpleMinAddressAddTest([Values] RevivifiableFraction revivifiableFr Populate(); // This should not go to FreeList because it's below the RevivifiableFraction - ClassicAssert.IsTrue(bContext.Delete(2).Found); + long keyNum = 2; + ClassicAssert.IsTrue(bContext.Delete(SpanByte.FromPinnedVariable(ref keyNum)).Found); ClassicAssert.AreEqual(0, RevivificationTestUtils.GetFreeRecordCount(store)); // This should go to FreeList because it's above the RevivifiableFraction - ClassicAssert.IsTrue(bContext.Delete(NumRecords - 1).Found); + keyNum = NumRecords - 1; + ClassicAssert.IsTrue(bContext.Delete(SpanByte.FromPinnedVariable(ref keyNum)).Found); ClassicAssert.AreEqual(1, RevivificationTestUtils.GetFreeRecordCount(store)); } @@ -412,7 +426,8 @@ public void SimpleMinAddressTakeTest([Values] RevivifiableFraction revivifiableF Populate(); // This should go to FreeList because it's above the RevivifiableFraction - ClassicAssert.IsTrue(bContext.Delete(NumRecords - 1).Found); + long keyNum = NumRecords - 1; + ClassicAssert.IsTrue(bContext.Delete(SpanByte.FromPinnedVariable(ref keyNum)).Found); ClassicAssert.AreEqual(1, RevivificationTestUtils.GetFreeRecordCount(store)); RevivificationTestUtils.WaitForRecords(store, want: true); @@ -420,10 +435,11 @@ public void SimpleMinAddressTakeTest([Values] RevivifiableFraction revivifiableF var pool = RevivificationTestUtils.SwapFreeRecordPool(store, default); // Now add a bunch of records to drop the FreeListed address below the RevivifiableFraction - int maxRecord = NumRecords * 2; - for (int key = NumRecords; key < maxRecord; key++) + long maxRecord = NumRecords * 2, valueNum; + for (keyNum = NumRecords; keyNum < maxRecord; keyNum++) { - var status = bContext.Upsert(key, key * ValueMult); + valueNum = keyNum * ValueMult; + var status = bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref valueNum)); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } @@ -431,7 +447,8 @@ public void SimpleMinAddressTakeTest([Values] RevivifiableFraction revivifiableF _ = RevivificationTestUtils.SwapFreeRecordPool(store, pool); var tailAddress = store.Log.TailAddress; - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(maxRecord, maxRecord * ValueMult) : bContext.RMW(maxRecord, maxRecord * ValueMult); + valueNum = maxRecord * ValueMult; + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(SpanByte.FromPinnedVariable(ref maxRecord), SpanByte.FromPinnedVariable(ref valueNum)) : bContext.RMW(SpanByte.FromPinnedVariable(ref maxRecord), ref valueNum); ClassicAssert.Less(tailAddress, store.Log.TailAddress, "Expected tail address to grow (record was not revivified)"); } @@ -449,15 +466,11 @@ class RevivificationSpanByteTests internal class RevivificationSpanByteFunctions : SpanByteFunctions { - private readonly TsavoriteKV> store; + private readonly TsavoriteKV> store; // Must be set after session is created - internal ClientSession> session; + internal ClientSession> session; - internal int expectedConcurrentDestLength = InitialLength; - internal int expectedSingleDestLength = InitialLength; - internal int expectedConcurrentFullValueLength = -1; - internal int expectedSingleFullValueLength = -1; internal int expectedInputLength = InitialLength; // used to configurably change RMW behavior to test tombstoning via RMW route. @@ -466,13 +479,13 @@ internal class RevivificationSpanByteFunctions : SpanByteFunctions internal bool deleteInCU = false; internal bool forceSkipIpu = false; - // This is a queue rather than a single value because there may be calls to, for example, ConcurrentWriter with one length - // followed by SingleWriter with another. - internal Queue expectedUsedValueLengths = new(); + // This is a queue rather than a single value because there may be calls to, for example, InPlaceWriter with one length + // followed by InitialWriter with another. + internal Queue expectedValueLengths = new(); internal bool readCcCalled, rmwCcCalled; - internal RevivificationSpanByteFunctions(TsavoriteKV> store) + internal RevivificationSpanByteFunctions(TsavoriteKV> store) { this.store = store; } @@ -490,70 +503,67 @@ private void AssertInfoValid(ref DeleteInfo deleteInfo) ClassicAssert.AreEqual(session.ctx.version, deleteInfo.Version); } - private static void VerifyKeyAndValue(ref SpanByte functionsKey, ref SpanByte functionsValue) + private static void VerifyKeyAndValue(ReadOnlySpan functionsKey, ReadOnlySpan functionsValue) { int valueOffset = 0, valueLengthRemaining = functionsValue.Length; ClassicAssert.Less(functionsKey.Length, valueLengthRemaining); while (valueLengthRemaining > 0) { var compareLength = Math.Min(functionsKey.Length, valueLengthRemaining); - Span valueSpan = functionsValue.AsSpan().Slice(valueOffset, compareLength); - Span keySpan = functionsKey.AsSpan()[..compareLength]; - ClassicAssert.IsTrue(valueSpan.SequenceEqual(keySpan), $"functionsValue (offset {valueOffset}, len {compareLength}: {SpanByte.FromPinnedSpan(valueSpan)}) does not match functionsKey ({SpanByte.FromPinnedSpan(keySpan)})"); + var valueSpan = functionsValue.Slice(valueOffset, compareLength); + var keySpan = functionsKey[..compareLength]; + ClassicAssert.IsTrue(valueSpan.SequenceEqual(keySpan), $"functionsValue (offset {valueOffset}, len {compareLength}: {valueSpan.ToShortString()}) does not match functionsKey ({keySpan.ToShortString()})"); valueLengthRemaining -= compareLength; } } - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + void CheckExpectedLengthsBefore(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, long recordAddress, bool isIPU = false) { - var rmwInfo = RevivificationTestUtils.CopyToRMWInfo(ref upsertInfo); - var result = InitialUpdater(ref key, ref input, ref dst, ref output, ref rmwInfo, ref recordInfo); - upsertInfo.UsedValueLength = rmwInfo.UsedValueLength; - return result; + var expectedValueLength = expectedValueLengths.Dequeue(); + + // 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)); + if (sizeInfo.ValueIsInline) + ClassicAssert.AreEqual(expectedValueLength, logRecord.ValueSpan.Length); + else + ClassicAssert.AreEqual(logRecord.Info.ValueIsInline ? expectedValueLength : ObjectIdMap.ObjectIdSize, logRecord.ValueSpan.Length); + + ClassicAssert.GreaterOrEqual(recordAddress, store.hlogBase.ReadOnlyAddress); + + // !IsSet means it is from Delete which does not receive a RecordSizeInfo. isIPU is an in-place update and thus the new value may legitimately be larger than the record. + if (sizeInfo.IsSet && !isIPU) + { + var (actual, allocated) = logRecord.GetInlineRecordSizes(); + ClassicAssert.AreEqual(sizeInfo.ActualInlineRecordSize, actual); + ClassicAssert.AreEqual(sizeInfo.AllocatedInlineRecordSize, allocated); + } } - public override bool ConcurrentWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - var rmwInfo = RevivificationTestUtils.CopyToRMWInfo(ref upsertInfo); - var result = InPlaceUpdater(ref key, ref input, ref dst, ref output, ref rmwInfo, ref recordInfo); - upsertInfo.UsedValueLength = rmwInfo.UsedValueLength; - return result; + CheckExpectedLengthsBefore(ref logRecord, ref sizeInfo, upsertInfo.Address); + return base.InitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - AssertInfoValid(ref rmwInfo); - ClassicAssert.AreEqual(expectedInputLength, input.Length); - - var expectedUsedValueLength = expectedUsedValueLengths.Dequeue(); - - if (value.Length == 0) - { - ClassicAssert.AreEqual(expectedUsedValueLength, rmwInfo.UsedValueLength); // for the length header - ClassicAssert.AreEqual(Constants.kRecordAlignment, rmwInfo.FullValueLength); // This should be the "added record for Delete" case, so a "default" value - } - else - { - ClassicAssert.AreEqual(expectedSingleDestLength, value.Length); - ClassicAssert.AreEqual(expectedSingleFullValueLength, rmwInfo.FullValueLength); - ClassicAssert.AreEqual(expectedUsedValueLength, rmwInfo.UsedValueLength); - ClassicAssert.GreaterOrEqual(rmwInfo.Address, store.hlogBase.ReadOnlyAddress); - } - return base.InitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + CheckExpectedLengthsBefore(ref logRecord, ref sizeInfo, upsertInfo.Address, isIPU: true); + return base.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override bool NeedCopyUpdate(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { if (deleteInNCU) { rmwInfo.Action = RMWAction.ExpireAndStop; return false; } - - return base.NeedCopyUpdate(ref key, ref input, ref oldValue, ref output, ref rmwInfo); + return base.NeedCopyUpdate(ref srcLogRecord, ref input, ref output, ref rmwInfo); } - public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { if (deleteInCU) { @@ -562,28 +572,13 @@ public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanB } AssertInfoValid(ref rmwInfo); - - ClassicAssert.AreEqual(expectedInputLength, input.Length); - var expectedUsedValueLength = expectedUsedValueLengths.Dequeue(); - - if (newValue.Length == 0) - { - ClassicAssert.AreEqual(sizeof(int), rmwInfo.UsedValueLength); // for the length header - ClassicAssert.AreEqual(Constants.kRecordAlignment, rmwInfo.FullValueLength); // This should be the "added record for Delete" case, so a "default" value - } - else - { - ClassicAssert.AreEqual(expectedSingleDestLength, newValue.Length); - ClassicAssert.AreEqual(expectedSingleFullValueLength, rmwInfo.FullValueLength); - ClassicAssert.AreEqual(expectedUsedValueLength, rmwInfo.UsedValueLength); - ClassicAssert.GreaterOrEqual(rmwInfo.Address, store.hlogBase.ReadOnlyAddress); - } - return base.CopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo, ref recordInfo); + CheckExpectedLengthsBefore(ref dstLogRecord, ref sizeInfo, rmwInfo.Address); + return dstLogRecord.TrySetValueSpan(input.ReadOnlySpan, ref sizeInfo); } - public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { AssertInfoValid(ref rmwInfo); @@ -597,105 +592,89 @@ public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref Sp } ClassicAssert.AreEqual(expectedInputLength, input.Length); - ClassicAssert.AreEqual(expectedConcurrentDestLength, value.Length); - ClassicAssert.AreEqual(expectedConcurrentFullValueLength, rmwInfo.FullValueLength); - - VerifyKeyAndValue(ref key, ref value); - var expectedUsedValueLength = expectedUsedValueLengths.Dequeue(); - ClassicAssert.AreEqual(expectedUsedValueLength, rmwInfo.UsedValueLength); + CheckExpectedLengthsBefore(ref logRecord, ref sizeInfo, rmwInfo.Address, isIPU: true); + VerifyKeyAndValue(logRecord.Key, logRecord.ValueSpan); - ClassicAssert.GreaterOrEqual(rmwInfo.Address, store.hlogBase.ReadOnlyAddress); - - return base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + return logRecord.TrySetValueSpan(input.ReadOnlySpan, ref sizeInfo); } - // Override the default SpanByteFunctions impelementation; for these tests, we always want the input length. - public override int GetRMWModifiedValueLength(ref SpanByte value, ref SpanByte input) => input.TotalSize; - - public override bool SingleDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public override bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { AssertInfoValid(ref deleteInfo); - ClassicAssert.AreEqual(expectedSingleDestLength, value.Length); - ClassicAssert.AreEqual(expectedSingleFullValueLength, deleteInfo.FullValueLength); - var expectedUsedValueLength = expectedUsedValueLengths.Dequeue(); - ClassicAssert.AreEqual(expectedUsedValueLength, deleteInfo.UsedValueLength); + RecordSizeInfo sizeInfo = default; + CheckExpectedLengthsBefore(ref logRecord, ref sizeInfo, deleteInfo.Address); - ClassicAssert.GreaterOrEqual(deleteInfo.Address, store.hlogBase.ReadOnlyAddress); - - return base.SingleDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + return base.InitialDeleter(ref logRecord, ref deleteInfo); } - public override bool ConcurrentDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public override bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { AssertInfoValid(ref deleteInfo); - ClassicAssert.AreEqual(expectedConcurrentDestLength, value.Length); - ClassicAssert.AreEqual(expectedConcurrentFullValueLength, deleteInfo.FullValueLength); - - var expectedUsedValueLength = expectedUsedValueLengths.Dequeue(); - ClassicAssert.AreEqual(expectedUsedValueLength, deleteInfo.UsedValueLength); - ClassicAssert.GreaterOrEqual(deleteInfo.Address, store.hlogBase.ReadOnlyAddress); + RecordSizeInfo sizeInfo = default; + CheckExpectedLengthsBefore(ref logRecord, ref sizeInfo, deleteInfo.Address); - return base.ConcurrentDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + return base.InPlaceDeleter(ref logRecord, ref deleteInfo); } - public override bool PostCopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public override bool PostCopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { AssertInfoValid(ref rmwInfo); - return base.PostCopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo); + return base.PostCopyUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); } - public override void PostInitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) + public override void PostInitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { AssertInfoValid(ref rmwInfo); - base.PostInitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo); + base.PostInitialUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); } - public override void PostSingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason writeReason) + public override void PostInitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { AssertInfoValid(ref upsertInfo); - base.PostSingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, writeReason); + base.PostInitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override void PostSingleDeleter(ref SpanByte key, ref DeleteInfo deleteInfo) + public override void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { AssertInfoValid(ref deleteInfo); - base.PostSingleDeleter(ref key, ref deleteInfo); + base.PostInitialDeleter(ref logRecord, ref deleteInfo); } - public override void ReadCompletionCallback(ref SpanByte key, ref SpanByte input, ref SpanByteAndMemory output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, Empty ctx, Status status, RecordMetadata recordMetadata) { readCcCalled = true; - base.ReadCompletionCallback(ref key, ref input, ref output, ctx, status, recordMetadata); + base.ReadCompletionCallback(ref diskLogRecord, ref input, ref output, ctx, status, recordMetadata); } - public override void RMWCompletionCallback(ref SpanByte key, ref SpanByte input, ref SpanByteAndMemory output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref PinnedSpanByte input, ref SpanByteAndMemory output, Empty ctx, Status status, RecordMetadata recordMetadata) { rmwCcCalled = true; - base.RMWCompletionCallback(ref key, ref input, ref output, ctx, status, recordMetadata); + base.RMWCompletionCallback(ref diskLogRecord, ref input, ref output, ctx, status, recordMetadata); } - } - static int RoundUpSpanByteFullValueLength(SpanByte input) => RoundupTotalSizeFullValue(input.TotalSize); - - static int RoundUpSpanByteFullValueLength(int dataLength) => RoundupTotalSizeFullValue(sizeof(int) + dataLength); - - internal static int RoundupTotalSizeFullValue(int length) => (length + Constants.kRecordAlignment - 1) & (~(Constants.kRecordAlignment - 1)); - - static int RoundUpSpanByteUsedLength(int dataLength) => RoundUp(SpanByteTotalSize(dataLength), sizeof(int)); - - static int SpanByteTotalSize(int dataLength) => sizeof(int) + dataLength; + // Override the default SpanByteFunctions impelementation; for these tests, we always want the input length. + /// + public override RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref PinnedSpanByte input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = input.Length }; + /// + public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref PinnedSpanByte input) + => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + /// + public override RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref PinnedSpanByte input) + => new() { KeyDataSize = key.Length, ValueDataSize = input.Length }; + } const int NumRecords = 200; RevivificationSpanByteFunctions functions; RevivificationSpanByteComparer comparer; - private TsavoriteKV> store; - private ClientSession> session; - private BasicContext> bContext; + private TsavoriteKV> store; + private ClientSession> session; + private BasicContext> bContext; private IDevice log; [SetUp] @@ -706,12 +685,13 @@ public void Setup() CollisionRange collisionRange = CollisionRange.None; - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = 1L << 24, LogDevice = log, PageSize = 1L << 17, MemorySize = 1L << 20, + MaxInlineValueSize = 1024, RevivificationSettings = RevivificationSettings.PowerOf2Bins }; @@ -737,12 +717,12 @@ public void Setup() comparer = new RevivificationSpanByteComparer(collisionRange); store = new(kvSettings - , StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) + , StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); functions = new RevivificationSpanByteFunctions(store); - session = store.NewSession(functions); + session = store.NewSession(functions); bContext = session.BasicContext; functions.session = session; } @@ -764,24 +744,20 @@ public void TearDown() void Populate(int from, int to) { - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(input); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory output = new(); for (int ii = from; ii < to; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(input.TotalSize); - var status = bContext.Upsert(ref key, ref input, ref input, ref output); + key.Fill((byte)ii); + input.Fill((byte)ii); + functions.expectedValueLengths.Enqueue(pinnedInputSpan.Length); + var status = bContext.Upsert(key, ref pinnedInputSpan, input, ref output); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); - ClassicAssert.IsEmpty(functions.expectedUsedValueLengths); + ClassicAssert.IsEmpty(functions.expectedValueLengths); } } @@ -794,10 +770,9 @@ public void SpanByteNoRevivLengthTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] Up { Populate(); - Span keyVec = stackalloc byte[KeyLength]; + Span key = stackalloc byte[KeyLength]; byte fillByte = 42; - keyVec.Fill(fillByte); - var key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); // Do NOT delete; this is a no-reviv test of lengths @@ -809,42 +784,30 @@ public void SpanByteNoRevivLengthTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] Up _ => -1 }; - functions.expectedSingleDestLength = functions.expectedInputLength; - functions.expectedConcurrentDestLength = InitialLength; // This is from the initial Populate() - functions.expectedSingleFullValueLength = RoundUpSpanByteFullValueLength(functions.expectedInputLength); - functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); - - Span inputVec = stackalloc byte[functions.expectedInputLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(fillByte); + Span input = stackalloc byte[functions.expectedInputLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(fillByte); // For Grow, we won't be able to satisfy the request with a revivification, and the new value length will be GrowLength - functions.expectedUsedValueLengths.Enqueue(sizeof(int) + InitialLength); + functions.expectedValueLengths.Enqueue(InitialLength); if (growth == Growth.Grow) - functions.expectedUsedValueLengths.Enqueue(sizeof(int) + GrowLength); + functions.expectedValueLengths.Enqueue(GrowLength); SpanByteAndMemory output = new(); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); - ClassicAssert.IsEmpty(functions.expectedUsedValueLengths); + ClassicAssert.IsEmpty(functions.expectedValueLengths); if (growth == Growth.Shrink) { - // What's there now will be what is passed to ConcurrentWriter/IPU (if Shrink, we kept the same value we allocated initially) - functions.expectedConcurrentFullValueLength = growth == Growth.Shrink ? RoundUpSpanByteFullValueLength(InitialLength) : functions.expectedSingleFullValueLength; - - // Now let's see if we have the correct expected extra length in the destination. - inputVec = stackalloc byte[InitialLength / 2]; // Grow this from ShrinkLength to InitialLength - input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(fillByte); + input = stackalloc byte[InitialLength / 2]; // Shrink this from InitialLength to ShrinkLength + input.Fill(fillByte); - functions.expectedInputLength = InitialLength / 2; - functions.expectedConcurrentDestLength = InitialLength / 2; - functions.expectedSingleFullValueLength = RoundUpSpanByteFullValueLength(functions.expectedInputLength); - functions.expectedUsedValueLengths.Enqueue(input.TotalSize); + functions.expectedInputLength = input.Length; + functions.expectedValueLengths.Enqueue(input.Length); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); - ClassicAssert.IsEmpty(functions.expectedUsedValueLengths); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); + ClassicAssert.IsEmpty(functions.expectedValueLengths); } } @@ -856,36 +819,36 @@ internal enum DeletionRoutes RMW_CU } - private Status DeleteViaRMW(ref SpanByte key, Span mockInputVec, byte fillByte) + private Status DeleteViaRMW(ReadOnlySpan key, Span mockInputVec, byte fillByte) { - var mockInput = SpanByte.FromPinnedSpan(mockInputVec); + var mockInput = PinnedSpanByte.FromPinnedSpan(mockInputVec); mockInputVec.Fill(fillByte); - return bContext.RMW(ref key, ref mockInput); + return bContext.RMW(key, ref mockInput); } - private Status PerformDeletion(DeletionRoutes deletionRoute, ref SpanByte key, byte fillByte) + private Status PerformDeletion(DeletionRoutes deletionRoute, ReadOnlySpan key, byte fillByte) { Status status; switch (deletionRoute) { case DeletionRoutes.DELETE: - return bContext.Delete(ref key); + return bContext.Delete(key); case DeletionRoutes.RMW_IPU: functions.deleteInIpu = true; Span mockInputVec = stackalloc byte[InitialLength]; - status = DeleteViaRMW(ref key, mockInputVec, fillByte); + status = DeleteViaRMW(key, mockInputVec, fillByte); functions.deleteInIpu = false; break; case DeletionRoutes.RMW_NCU: functions.deleteInNCU = true; mockInputVec = stackalloc byte[InitialLength]; - status = DeleteViaRMW(ref key, mockInputVec, fillByte); + status = DeleteViaRMW(key, mockInputVec, fillByte); functions.deleteInNCU = false; break; case DeletionRoutes.RMW_CU: functions.deleteInCU = true; mockInputVec = stackalloc byte[InitialLength]; - status = DeleteViaRMW(ref key, mockInputVec, fillByte); + status = DeleteViaRMW(key, mockInputVec, fillByte); functions.deleteInCU = false; break; default: @@ -904,34 +867,29 @@ public void SpanByteSimpleTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp var tailAddress = store.Log.TailAddress; - Span keyVec = stackalloc byte[KeyLength]; + Span key = stackalloc byte[KeyLength]; byte fillByte = 42; - keyVec.Fill(fillByte); - var key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - - Status status = PerformDeletion(deletionRoute, ref key, fillByte); + functions.expectedValueLengths.Enqueue(InitialLength); + Status status = PerformDeletion(deletionRoute, key, fillByte); ClassicAssert.IsTrue(status.Found, status.ToString()); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(fillByte); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(fillByte); SpanByteAndMemory output = new(); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); RevivificationTestUtils.WaitForRecords(store, want: true); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); } @@ -943,15 +901,14 @@ public void SpanByteDeletionViaRMWRCURevivifiesOriginalRecordAfterTombstoning( { Populate(); - Span keyVec = stackalloc byte[KeyLength]; + Span key = stackalloc byte[KeyLength]; byte fillByte = 42; - keyVec.Fill(fillByte); - var key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); functions.forceSkipIpu = true; - var status = PerformDeletion(deletionRoute, ref key, fillByte); + var status = PerformDeletion(deletionRoute, key, fillByte); functions.forceSkipIpu = false; RevivificationTestUtils.WaitForRecords(store, want: true); @@ -963,24 +920,19 @@ public void SpanByteDeletionViaRMWRCURevivifiesOriginalRecordAfterTombstoning( var tailAddress = store.Log.TailAddress; Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + var input = PinnedSpanByte.FromPinnedSpan(inputVec); inputVec.Fill(fillByte); SpanByteAndMemory output = new(); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(input); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); // brand new value so we try to use a record out of free list - keyVec = stackalloc byte[KeyLength]; fillByte = 255; - keyVec.Fill(fillByte); - key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref input, input.ReadOnlySpan, ref output) : bContext.RMW(key, ref input); // since above would use revivification free list we should see no change of tail address. ClassicAssert.AreEqual(store.Log.TailAddress, tailAddress); @@ -998,35 +950,30 @@ public void SpanByteIPUGrowAndRevivifyTest([Values(UpdateOp.Upsert, UpdateOp.RMW var tailAddress = store.Log.TailAddress; - Span keyVec = stackalloc byte[KeyLength]; + Span key = stackalloc byte[KeyLength]; byte fillByte = 42; - keyVec.Fill(fillByte); - var key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); - Span inputVec = stackalloc byte[GrowLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(fillByte); + Span input = stackalloc byte[GrowLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(fillByte); SpanByteAndMemory output = new(); functions.expectedInputLength = GrowLength; - functions.expectedSingleDestLength = GrowLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = RoundUpSpanByteFullValueLength(GrowLength); - functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(GrowLength)); + functions.expectedValueLengths.Enqueue(InitialLength); + functions.expectedValueLengths.Enqueue(GrowLength); // Get a free record from a failed IPU. if (updateOp == UpdateOp.Upsert) { - var status = bContext.Upsert(ref key, ref input, ref input, ref output); + var status = bContext.Upsert(key, ref pinnedInputSpan, input, ref output); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } else if (updateOp == UpdateOp.RMW) { - var status = bContext.RMW(ref key, ref input); + var status = bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.IsTrue(status.Record.CopyUpdated, status.ToString()); } @@ -1039,23 +986,21 @@ public void SpanByteIPUGrowAndRevivifyTest([Values(UpdateOp.Upsert, UpdateOp.RMW RevivificationTestUtils.WaitForRecords(store, want: true); // Get a new key and shrink the requested length so we revivify the free record from the failed IPU. - keyVec.Fill(NumRecords + 1); - input = SpanByte.FromPinnedSpan(inputVec.Slice(0, InitialLength)); + key.Fill(NumRecords + 1); + input = input.Slice(0, InitialLength); + pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); if (updateOp == UpdateOp.Upsert) { - var status = bContext.Upsert(ref key, ref input, ref input, ref output); + var status = bContext.Upsert(key, ref pinnedInputSpan, input, ref output); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } else if (updateOp == UpdateOp.RMW) { - var status = bContext.RMW(ref key, ref input); + var status = bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } @@ -1073,31 +1018,27 @@ public void SpanByteReadOnlyMinAddressTest([Values(UpdateOp.Upsert, UpdateOp.RMW var tailAddress = store.Log.TailAddress; - Span keyVec = stackalloc byte[KeyLength]; + Span key = stackalloc byte[KeyLength]; byte fillByte = 42; - keyVec.Fill(fillByte); - var key = SpanByte.FromPinnedSpan(keyVec); + key.Fill(fillByte); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, wait: true); - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(fillByte); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(fillByte); SpanByteAndMemory output = new(); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(input); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.Greater(store.Log.TailAddress, tailAddress); } @@ -1114,12 +1055,11 @@ private long PrepareDeletes(bool stayInChain, byte delAboveRO, FlushMode flushMo var pool = stayInChain ? RevivificationTestUtils.SwapFreeRecordPool(store, null) : null; // Delete key below (what will be) the readonly line. This is for a target for the test; the record should not be revivified. - Span keyVecDelBelowRO = stackalloc byte[KeyLength]; - keyVecDelBelowRO.Fill(DelBelowRO); - var delKeyBelowRO = SpanByte.FromPinnedSpan(keyVecDelBelowRO); + Span delKeyBelowRO = stackalloc byte[KeyLength]; + delKeyBelowRO.Fill(DelBelowRO); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref delKeyBelowRO); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(delKeyBelowRO); ClassicAssert.IsTrue(status.Found, status.ToString()); if (flushMode == FlushMode.ReadOnly) @@ -1133,15 +1073,14 @@ private long PrepareDeletes(bool stayInChain, byte delAboveRO, FlushMode flushMo // Delete key above the readonly line. This is the record that will be revivified. // If not stayInChain, this also puts two elements in the free list; one should be skipped over on Take() as it is below readonly. - Span keyVecDelAboveRO = stackalloc byte[KeyLength]; - keyVecDelAboveRO.Fill(delAboveRO); - var delKeyAboveRO = SpanByte.FromPinnedSpan(keyVecDelAboveRO); + Span delKeyAboveRO = stackalloc byte[KeyLength]; + delKeyAboveRO.Fill(delAboveRO); if (!stayInChain && collisionRange == CollisionRange.None) // CollisionRange.Ten has a valid .PreviousAddress so won't be moved to FreeList - RevivificationTestUtils.AssertElidable(store, ref delKeyAboveRO); + RevivificationTestUtils.AssertElidable(store, delKeyAboveRO); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - status = bContext.Delete(ref delKeyAboveRO); + functions.expectedValueLengths.Enqueue(InitialLength); + status = bContext.Delete(delKeyAboveRO); ClassicAssert.IsTrue(status.Found, status.ToString()); if (stayInChain) @@ -1155,11 +1094,6 @@ private long PrepareDeletes(bool stayInChain, byte delAboveRO, FlushMode flushMo } ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); - - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); - return tailAddress; } @@ -1181,13 +1115,12 @@ public void SpanByteUpdateRevivifyTest([Values] DeleteDest deleteDest, [Values] long tailAddress = PrepareDeletes(stayInChain, delAboveRO, FlushMode.ReadOnly, collisionRange); - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory output = new(); - Span keyVecToTest = stackalloc byte[KeyLength]; - var keyToTest = SpanByte.FromPinnedSpan(keyVecToTest); + Span keyToTest = stackalloc byte[KeyLength]; bool expectReviv; if (updateKey is UpdateKey.Unfound or UpdateKey.CopiedBelowRO) @@ -1197,8 +1130,8 @@ public void SpanByteUpdateRevivifyTest([Values] DeleteDest deleteDest, [Values] // (but exercises a different code path than Unfound). // CollisionRange.Ten has a valid PreviousAddress so it is not elided from the cache. byte fillByte = updateKey == UpdateKey.Unfound ? Unfound : CopiedBelowRO; - keyVecToTest.Fill(fillByte); - inputVec.Fill(fillByte); + keyToTest.Fill(fillByte); + input.Fill(fillByte); expectReviv = !stayInChain && collisionRange != CollisionRange.Ten; } else if (updateKey == UpdateKey.DeletedBelowRO) @@ -1206,16 +1139,16 @@ public void SpanByteUpdateRevivifyTest([Values] DeleteDest deleteDest, [Values] // DeletedBelowRO will not match the key for the in-chain above-RO slot, and we cannot reviv below RO or retrieve below-RO from the // freelist, so we will always allocate a new record unless we're using the freelist. byte fillByte = DelBelowRO; - keyVecToTest.Fill(fillByte); - inputVec.Fill(fillByte); + keyToTest.Fill(fillByte); + input.Fill(fillByte); expectReviv = !stayInChain && collisionRange != CollisionRange.Ten; } else if (updateKey == UpdateKey.DeletedAboveRO) { // DeletedAboveRO means we will reuse an in-chain record, or will get it from the freelist if deleteDest is FreeList. byte fillByte = delAboveRO; - keyVecToTest.Fill(fillByte); - inputVec.Fill(fillByte); + keyToTest.Fill(fillByte); + input.Fill(fillByte); expectReviv = true; } else @@ -1225,13 +1158,9 @@ public void SpanByteUpdateRevivifyTest([Values] DeleteDest deleteDest, [Values] } functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - - if (!expectReviv) - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(input); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref keyToTest, ref input, ref input, ref output) : bContext.RMW(ref keyToTest, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(keyToTest, ref pinnedInputSpan, input, ref output) : bContext.RMW(keyToTest, ref pinnedInputSpan); if (expectReviv) ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); @@ -1252,21 +1181,20 @@ public void SimpleRevivifyTest([Values] DeleteDest deleteDest, [Values(UpdateOp. // This freed record stays in the hash chain. byte chainKey = NumRecords / 2 - 1; - Span keyVec = stackalloc byte[KeyLength]; - keyVec.Fill(chainKey); - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; + key.Fill(chainKey); if (!stayInChain) - RevivificationTestUtils.AssertElidable(store, ref key); + RevivificationTestUtils.AssertElidable(store, key); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); var tailAddress = store.Log.TailAddress; - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(chainKey); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(chainKey); SpanByteAndMemory output = new(); @@ -1274,8 +1202,8 @@ public void SimpleRevivifyTest([Values] DeleteDest deleteDest, [Values(UpdateOp. RevivificationTestUtils.WaitForRecords(store, want: true); // Revivify in the chain. Because this stays in the chain, the expectedFullValueLength is roundup(InitialLength) - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + functions.expectedValueLengths.Enqueue(InitialLength); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); } @@ -1289,20 +1217,19 @@ public void DeleteEntireChainAndRevivifyTest([Values(CollisionRange.Ten)] Collis // These freed records stay in the hash chain; we even skip the first one to ensure nothing goes into the free list. byte chainKey = 5; - Span keyVec = stackalloc byte[KeyLength]; - keyVec.Fill(chainKey); - var key = SpanByte.FromPinnedSpan(keyVec); - var hash = comparer.GetHashCode64(ref key); + Span key = stackalloc byte[KeyLength]; + key.Fill(chainKey); + var hash = comparer.GetHashCode64(key); List deletedSlots = []; for (int ii = chainKey + 1; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - if (comparer.GetHashCode64(ref key) != hash) + key.Fill((byte)ii); + if (comparer.GetHashCode64(key) != hash) continue; - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); if (ii > RevivificationTestUtils.GetMinRevivifiableKey(store, NumRecords)) deletedSlots.Add((byte)ii); @@ -1315,19 +1242,19 @@ public void DeleteEntireChainAndRevivifyTest([Values(CollisionRange.Ten)] Collis ClassicAssert.Greater(deletedSlots.Count, 5); // should be about Ten var tailAddress = store.Log.TailAddress; - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); - inputVec.Fill(chainKey); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); + input.Fill(chainKey); SpanByteAndMemory output = new(); // Revivify in the chain. Because this stays in the chain, the expectedFullValueLength is roundup(InitialLength) for (int ii = 0; ii < deletedSlots.Count; ++ii) { - keyVec.Fill(deletedSlots[ii]); + key.Fill(deletedSlots[ii]); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + functions.expectedValueLengths.Enqueue(InitialLength); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); } } @@ -1341,51 +1268,46 @@ public void DeleteAllRecordsAndRevivifyTest([Values(CollisionRange.None)] Collis long tailAddress = store.Log.TailAddress; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; // "sizeof(int) +" because SpanByte has an int length prefix - var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + keyVec.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); + var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + key.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); // Delete for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); + key.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); } ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); ClassicAssert.AreEqual(RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords), RevivificationTestUtils.GetFreeRecordCount(store), $"Expected numRecords ({NumRecords}) free records"); - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory output = new(); - functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - // These come from the existing initial allocation so keep the full length - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); + functions.expectedInputLength = InitialLength; // Revivify var revivifiableKeyCount = RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords); for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + functions.expectedValueLengths.Enqueue(InitialLength); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); if (ii < revivifiableKeyCount) ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress, $"unexpected new record for key {ii}"); else ClassicAssert.Less(tailAddress, store.Log.TailAddress, $"unexpected revivified record for key {ii}"); - var status = bContext.Read(ref key, ref output); + var status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.Found, $"Expected to find key {ii}; status == {status}"); } @@ -1395,8 +1317,8 @@ public void DeleteAllRecordsAndRevivifyTest([Values(CollisionRange.None)] Collis // Confirm for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - var status = bContext.Read(ref key, ref output); + key.Fill((byte)ii); + var status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.Found, $"Expected to find key {ii}; status == {status}"); } } @@ -1408,16 +1330,15 @@ public void DeleteAllRecordsAndTakeSnapshotTest() { Populate(); - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; // Delete for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); + key.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); } ClassicAssert.AreEqual(RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords), RevivificationTestUtils.GetFreeRecordCount(store), $"Expected numRecords ({NumRecords}) free records"); @@ -1432,24 +1353,23 @@ public void DeleteAllRecordsAndIterateTest() { Populate(); - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; // Delete for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); + key.Fill((byte)ii); - RevivificationTestUtils.AssertElidable(store, ref key); + RevivificationTestUtils.AssertElidable(store, key); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); } ClassicAssert.AreEqual(RevivificationTestUtils.GetRevivifiableRecordCount(store, NumRecords), RevivificationTestUtils.GetFreeRecordCount(store), $"Expected numRecords ({NumRecords}) free records"); using var iterator = session.Iterate(); - while (iterator.GetNext(out _)) + while (iterator.GetNext()) ; } @@ -1479,56 +1399,6 @@ public void BinSelectionTest() } } - [Test] - [Category(RevivificationCategory)] - [Category(SmokeTestCategory)] - //[Repeat(30)] - public unsafe void ArtificialBinWrappingTest() - { - var pool = store.RevivificationManager.FreeRecordPool; - - if (TestContext.CurrentContext.CurrentRepeatCount > 0) - Debug.WriteLine($"*** Current test iteration: {TestContext.CurrentContext.CurrentRepeatCount + 1} ***"); - - Populate(); - - const int recordSize = 42; - - ClassicAssert.IsTrue(pool.GetBinIndex(recordSize, out int binIndex)); - ClassicAssert.AreEqual(2, binIndex); - - const int minAddress = 1_000; - int logicalAddress = 1_000_000; - - RevivificationStats revivStats = new(); - - // Fill the bin, including wrapping around at the end. - var recordCount = RevivificationTestUtils.GetRecordCount(pool, binIndex); - for (var ii = 0; ii < recordCount; ++ii) - ClassicAssert.IsTrue(store.RevivificationManager.TryAdd(logicalAddress + ii, recordSize, ref revivStats), "ArtificialBinWrappingTest: Failed to Add free record, pt 1"); - - // Try to add to a full bin; this should fail. - revivStats.Reset(); - ClassicAssert.IsFalse(store.RevivificationManager.TryAdd(logicalAddress + recordCount, recordSize, ref revivStats), "ArtificialBinWrappingTest: Expected to fail Adding free record"); - - RevivificationTestUtils.WaitForRecords(store, want: true); - - for (var ii = 0; ii < recordCount; ++ii) - ClassicAssert.IsTrue(RevivificationTestUtils.IsSet(pool, binIndex, ii), "expected bin to be set at ii == {ii}"); - - // Take() one to open up a space in the bin, then add one - revivStats.Reset(); - ClassicAssert.IsTrue(RevivificationTestUtils.TryTakeFromBin(pool, binIndex, recordSize, minAddress, store, out _, ref revivStats)); - revivStats.Reset(); - ClassicAssert.IsTrue(store.RevivificationManager.TryAdd(logicalAddress + recordCount + 1, recordSize, ref revivStats), "ArtificialBinWrappingTest: Failed to Add free record, pt 2"); - - // Take() all records in the bin. - revivStats.Reset(); - for (var ii = 0; ii < recordCount; ++ii) - ClassicAssert.IsTrue(RevivificationTestUtils.TryTakeFromBin(pool, binIndex, recordSize, minAddress, store, out _, ref revivStats), $"ArtificialBinWrappingTest: failed to Take at ii == {ii}"); - _ = revivStats.Dump(); - } - [Test] [Category(RevivificationCategory)] [Category(SmokeTestCategory)] @@ -1544,14 +1414,12 @@ public unsafe void LiveBinWrappingTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U var pool = store.RevivificationManager.FreeRecordPool; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); // "sizeof(int) +" because SpanByte has an int length prefix. - var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + keyVec.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); + var recordSize = RecordInfo.GetLength() + RoundUp(sizeof(int) + key.Length, 8) + RoundUp(sizeof(int) + InitialLength, 8); ClassicAssert.IsTrue(pool.GetBinIndex(recordSize, out int binIndex)); ClassicAssert.AreEqual(3, binIndex); @@ -1562,11 +1430,11 @@ public unsafe void LiveBinWrappingTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U functions.expectedInputLength = InitialLength; for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, $"{status} for key {ii}"); //ClassicAssert.AreEqual(ii + 1, RevivificationTestUtils.GetFreeRecordCount(store), $"mismatched free record count for key {ii}, pt 1"); } @@ -1579,18 +1447,16 @@ public unsafe void LiveBinWrappingTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U // Revivify functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); long tailAddress = store.Log.TailAddress; SpanByteAndMemory output = new(); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); output.Memory?.Dispose(); if (deleteDest == DeleteDest.FreeList && waitMode == WaitMode.Wait && tailAddress != store.Log.TailAddress) @@ -1622,11 +1488,9 @@ public void LiveBinWrappingNoRevivTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U // For a comparison to the reviv version above. Populate(); - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); for (var iter = 0; iter < 100; ++iter) { @@ -1634,23 +1498,23 @@ public void LiveBinWrappingNoRevivTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U functions.expectedInputLength = InitialLength; for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(iter == 0 ? InitialLength : InitialLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(iter == 0 ? InitialLength : InitialLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, $"{status} for key {ii}, iter {iter}"); } for (var ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); SpanByteAndMemory output = new(); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); output.Memory?.Dispose(); } } @@ -1659,8 +1523,9 @@ public void LiveBinWrappingNoRevivTest([Values(UpdateOp.Upsert, UpdateOp.RMW)] U [Test] [Category(RevivificationCategory)] [Category(SmokeTestCategory)] - public void SimpleOversizeRevivifyTest([Values] DeleteDest deleteDest, [Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp updateOp) + public void SimpleOverflowRevivifyTest([Values] DeleteDest deleteDest, [Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp updateOp) { + Assert.Ignore("Test ignored because SpanByteAllocatore currently does not support overflow."); Populate(); bool stayInChain = deleteDest == DeleteDest.InChain; @@ -1670,30 +1535,27 @@ public void SimpleOversizeRevivifyTest([Values] DeleteDest deleteDest, [Values(U _ = RevivificationTestUtils.SwapFreeRecordPool(store, default); byte chainKey = NumRecords + 1; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; - Span inputVec = stackalloc byte[OversizeLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[OversizeLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory output = new(); - keyVec.Fill(chainKey); - inputVec.Fill(chainKey); + key.Fill(chainKey); + input.Fill(chainKey); // Oversize records in this test do not go to "next higher" bin (there is no next-higher bin in the default PowersOf2 bins we use) + // and they become an out-of-line pointer. functions.expectedInputLength = OversizeLength; - functions.expectedSingleDestLength = OversizeLength; - functions.expectedConcurrentDestLength = OversizeLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(OversizeLength); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(OversizeLength)); + functions.expectedValueLengths.Enqueue(ObjectIdMap.ObjectIdSize); // Initial insert of the oversize record - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); // Delete it - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(OversizeLength)); - var status = bContext.Delete(ref key); + functions.expectedValueLengths.Enqueue(OversizeLength); + var status = bContext.Delete(key); ClassicAssert.IsTrue(status.Found, status.ToString()); if (!stayInChain) RevivificationTestUtils.WaitForRecords(store, want: true); @@ -1701,8 +1563,8 @@ public void SimpleOversizeRevivifyTest([Values] DeleteDest deleteDest, [Values(U var tailAddress = store.Log.TailAddress; // Revivify in the chain. Because this is oversize, the expectedFullValueLength remains the same - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(OversizeLength)); - _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(ref key, ref input, ref input, ref output) : bContext.RMW(ref key, ref input); + functions.expectedValueLengths.Enqueue(OversizeLength); + _ = updateOp == UpdateOp.Upsert ? bContext.Upsert(key, ref pinnedInputSpan, input, ref output) : bContext.RMW(key, ref pinnedInputSpan); ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress); } @@ -1726,46 +1588,38 @@ public void SimplePendingOpsRevivifyTest([Values(CollisionRange.None)] Collision SpanByteAndMemory output = new(); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - functions.expectedSingleFullValueLength = functions.expectedConcurrentFullValueLength = RoundUpSpanByteFullValueLength(InitialLength); // Use a different key below RO than we deleted; this will go pending to retrieve it - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; if (pendingOp == PendingOp.Read) { - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[InitialLength]; - keyVec.Fill(targetRO); - inputVec.Fill(targetRO); + key.Fill(targetRO); + input.Fill(targetRO); functions.expectedInputLength = InitialLength; - functions.expectedSingleDestLength = InitialLength; - functions.expectedConcurrentDestLength = InitialLength; - var spanSlice = inputVec[..InitialLength]; - var inputSlice = SpanByte.FromPinnedSpan(spanSlice); + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input[..InitialLength]); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); - var status = bContext.Read(ref key, ref inputSlice, ref output); + functions.expectedValueLengths.Enqueue(InitialLength); + var status = bContext.Read(key, ref pinnedInputSpan, ref output); ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = bContext.CompletePending(wait: true); ClassicAssert.IsTrue(functions.readCcCalled); } else if (pendingOp == PendingOp.RMW) { - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); - keyVec.Fill(targetRO); - inputVec.Fill(targetRO); + key.Fill(targetRO); + input.Fill(targetRO); - functions.expectedUsedValueLengths.Enqueue(SpanByteTotalSize(InitialLength)); + functions.expectedValueLengths.Enqueue(InitialLength); - _ = bContext.RMW(ref key, ref input); + _ = bContext.RMW(key, ref pinnedInputSpan); _ = bContext.CompletePending(wait: true); ClassicAssert.IsTrue(functions.rmwCcCalled); } @@ -1773,6 +1627,7 @@ public void SimplePendingOpsRevivifyTest([Values(CollisionRange.None)] Collision } } +#if LOGRECORD_TODO [TestFixture] class RevivificationObjectTests { @@ -1864,6 +1719,7 @@ public void SimpleObjectTest([Values] DeleteDest deleteDest, [Values(UpdateOp.Up ClassicAssert.AreEqual(tailAddress, store.Log.TailAddress, "Expected tail address not to grow (record was revivified)"); } } +#endif // LOGRECORD_TODO [TestFixture] class RevivificationSpanByteStressTests @@ -1873,23 +1729,23 @@ class RevivificationSpanByteStressTests internal class RevivificationStressFunctions : SpanByteFunctions { - internal IKeyComparer keyComparer; // non-null if we are doing key comparisons (and thus expectedKey is non-default) - internal SpanByte expectedKey = default; // Set for each operation by the calling thread + internal IKeyComparer keyComparer; // non-null if we are doing key comparisons (and thus expectedKey is non-default) + internal PinnedSpanByte expectedKey = default; // Set for each operation by the calling thread internal bool isFirstLap = true; // For first - internal RevivificationStressFunctions(IKeyComparer keyComparer) => this.keyComparer = keyComparer; + internal RevivificationStressFunctions(IKeyComparer keyComparer) => this.keyComparer = keyComparer; [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void VerifyKey(ref SpanByte functionsKey) + private void VerifyKey(ReadOnlySpan functionsKey) { if (keyComparer is not null) - ClassicAssert.IsTrue(keyComparer.Equals(ref expectedKey, ref functionsKey)); + ClassicAssert.IsTrue(keyComparer.Equals(expectedKey.ReadOnlySpan, functionsKey)); } - private void VerifyKeyAndValue(ref SpanByte functionsKey, ref SpanByte functionsValue) + private void VerifyKeyAndValue(ReadOnlySpan functionsKey, ReadOnlySpan functionsValue) { if (keyComparer is not null) - ClassicAssert.IsTrue(keyComparer.Equals(ref expectedKey, ref functionsKey), "functionsKey does not equal expectedKey"); + ClassicAssert.IsTrue(keyComparer.Equals(expectedKey.ReadOnlySpan, functionsKey), "functionsKey does not equal expectedKey"); // Even in CompletePending(), we can verify internal consistency of key/value int valueOffset = 0, valueLengthRemaining = functionsValue.Length; @@ -1897,49 +1753,49 @@ private void VerifyKeyAndValue(ref SpanByte functionsKey, ref SpanByte functions while (valueLengthRemaining > 0) { var compareLength = Math.Min(functionsKey.Length, valueLengthRemaining); - Span valueSpan = functionsValue.AsSpan().Slice(valueOffset, compareLength); - Span keySpan = functionsKey.AsSpan()[..compareLength]; - ClassicAssert.IsTrue(valueSpan.SequenceEqual(keySpan), $"functionsValue (offset {valueOffset}, len {compareLength}: {SpanByte.FromPinnedSpan(valueSpan)}) does not match functionsKey ({SpanByte.FromPinnedSpan(keySpan)})"); + var valueSpan = functionsValue.Slice(valueOffset, compareLength); + var keySpan = functionsKey[..compareLength]; + ClassicAssert.IsTrue(valueSpan.SequenceEqual(keySpan), $"functionsValue (offset {valueOffset}, len {compareLength}: {valueSpan.ToShortString()}) does not match functionsKey ({keySpan.ToShortString()})"); valueOffset += compareLength; valueLengthRemaining -= compareLength; } } - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - VerifyKey(ref key); - return base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + VerifyKey(logRecord.Key); + return base.InitialWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override bool ConcurrentWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ReadOnlySpan srcValue, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo) { - VerifyKeyAndValue(ref key, ref dst); - return base.ConcurrentWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo); + VerifyKeyAndValue(logRecord.Key, srcValue); + return base.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); } - public override bool InitialUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - VerifyKey(ref key); - return base.InitialUpdater(ref key, ref input, ref newValue, ref output, ref rmwInfo, ref recordInfo); + VerifyKey(logRecord.Key); + return logRecord.TrySetValueSpan(input.ReadOnlySpan, ref sizeInfo); } - public override bool CopyUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte oldValue, ref SpanByte newValue, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - VerifyKeyAndValue(ref key, ref oldValue); - return base.CopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo, ref recordInfo); + VerifyKeyAndValue(srcLogRecord.Key, srcLogRecord.ValueSpan); + return dstLogRecord.TrySetValueSpan(srcLogRecord.ValueSpan, ref sizeInfo); } - public override bool InPlaceUpdater(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref SpanByteAndMemory output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref PinnedSpanByte input, ref SpanByteAndMemory output, ref RMWInfo rmwInfo) { - VerifyKeyAndValue(ref key, ref value); - return base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + VerifyKeyAndValue(logRecord.Key, logRecord.ValueSpan); + return logRecord.TrySetValueSpan(input.ReadOnlySpan, ref sizeInfo); } - public override bool SingleDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) - => base.SingleDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + public override bool InitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + => base.InitialDeleter(ref logRecord, ref deleteInfo); - public override unsafe bool ConcurrentDeleter(ref SpanByte key, ref SpanByte value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) - => base.ConcurrentDeleter(ref key, ref value, ref deleteInfo, ref recordInfo); + public override unsafe bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) + => base.InPlaceDeleter(ref logRecord, ref deleteInfo); } const int NumRecords = 200; @@ -1948,9 +1804,9 @@ public override unsafe bool ConcurrentDeleter(ref SpanByte key, ref SpanByte val RevivificationStressFunctions functions; RevivificationSpanByteComparer comparer; - private TsavoriteKV> store; - private ClientSession> session; - private BasicContext> bContext; + private TsavoriteKV> store; + private ClientSession> session; + private BasicContext> bContext; private IDevice log; [SetUp] @@ -1977,12 +1833,12 @@ public void Setup() PageSize = 1L << 17, MemorySize = 1L << 20, RevivificationSettings = RevivificationSettings.PowerOf2Bins - }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) + }, StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); functions = new RevivificationStressFunctions(keyComparer: null); - session = store.NewSession(functions); + session = store.NewSession(functions); bContext = session.BasicContext; } @@ -2001,26 +1857,25 @@ public void TearDown() unsafe void Populate() { - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); SpanByteAndMemory output = new(); for (int ii = 0; ii < NumRecords; ++ii) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - var status = bContext.Upsert(ref key, ref input, ref input, ref output); + var status = bContext.Upsert(key, ref pinnedInputSpan, input, ref output); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); } } const int AddressIncrement = 1_000_000; // must be > ReadOnlyAddress +#if LOGRECORD_TODO // Artificial bins - switch to SpanByte to ensure First/Best fit are tested [Test] [Category(RevivificationCategory)] [TestCase(20, 1, 1)] @@ -2333,6 +2188,7 @@ unsafe void runThread(int tid) ClassicAssert.IsTrue(counter == 0); } +#endif // LOGRECORD_TODO // Artificial bins - switch to SpanByte to ensure First/Best fit are tested [Test] [Category(RevivificationCategory)] @@ -2350,18 +2206,17 @@ unsafe void runDeleteThread(int tid) { Random rng = new(tid * 101); - using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); + using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); var localbContext = localSession.BasicContext; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; for (var iteration = 0; iteration < numIterations; ++iteration) { for (var ii = tid; ii < NumRecords; ii += numDeleteThreads) { var kk = rng.Next(keyRange); - keyVec.Fill((byte)kk); + key.Fill((byte)kk); _ = localbContext.Delete(key); } } @@ -2369,16 +2224,15 @@ unsafe void runDeleteThread(int tid) unsafe void runUpdateThread(int tid) { - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); Random rng = new(tid * 101); RevivificationStressFunctions localFunctions = new(keyComparer: comparer); - using var localSession = store.NewSession(localFunctions); + using var localSession = store.NewSession(localFunctions); var localbContext = localSession.BasicContext; for (var iteration = 0; iteration < numIterations; ++iteration) @@ -2386,11 +2240,11 @@ unsafe void runUpdateThread(int tid) for (var ii = tid; ii < NumRecords; ii += numUpdateThreads) { var kk = rng.Next(keyRange); - keyVec.Fill((byte)kk); - inputVec.Fill((byte)kk); + key.Fill((byte)kk); + input.Fill((byte)kk); - localSession.functions.expectedKey = key; - _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, input); + localSession.functions.expectedKey = PinnedSpanByte.FromPinnedSpan(key); + _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, ref pinnedInputSpan); localSession.functions.expectedKey = default; } @@ -2433,18 +2287,17 @@ unsafe void runDeleteThread(int tid) { Random rng = new(tid * 101); - using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); + using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); var localbContext = localSession.BasicContext; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; for (var iteration = 0; iteration < numIterations; ++iteration) { for (var ii = tid; ii < NumRecords; ii += numDeleteThreads) { var kk = threadingPattern == ThreadingPattern.RandomKeys ? rng.Next(NumRecords) : ii; - keyVec.Fill((byte)kk); + key.Fill((byte)kk); _ = localbContext.Delete(key); } } @@ -2452,16 +2305,14 @@ unsafe void runDeleteThread(int tid) unsafe void runUpdateThread(int tid) { - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); Random rng = new(tid * 101); RevivificationStressFunctions localFunctions = new(keyComparer: comparer); - using var localSession = store.NewSession(localFunctions); + using var localSession = store.NewSession(localFunctions); var localbContext = localSession.BasicContext; for (var iteration = 0; iteration < numIterations; ++iteration) @@ -2469,11 +2320,11 @@ unsafe void runUpdateThread(int tid) for (var ii = tid; ii < NumRecords; ii += numUpdateThreads) { var kk = threadingPattern == ThreadingPattern.RandomKeys ? rng.Next(NumRecords) : ii; - keyVec.Fill((byte)kk); - inputVec.Fill((byte)kk); + key.Fill((byte)kk); + input.Fill((byte)kk); - localSession.functions.expectedKey = key; - _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, input); + localSession.functions.expectedKey = PinnedSpanByte.FromPinnedSpan(key); + _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, ref pinnedInputSpan); localSession.functions.expectedKey = default; } @@ -2514,17 +2365,16 @@ public void LiveInChainThreadStressTest([Values(CollisionRange.Ten)] CollisionRa unsafe void runDeleteThread(int tid) { - using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); + using var localSession = store.NewSession(new RevivificationStressFunctions(keyComparer: null)); var localbContext = localSession.BasicContext; - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); + Span key = stackalloc byte[KeyLength]; for (var iteration = 0; iteration < numIterations; ++iteration) { for (var ii = tid; ii < NumRecords; ii += numDeleteThreads) { - keyVec.Fill((byte)ii); + key.Fill((byte)ii); _ = localbContext.Delete(key); } } @@ -2532,25 +2382,23 @@ unsafe void runDeleteThread(int tid) unsafe void runUpdateThread(int tid) { - Span keyVec = stackalloc byte[KeyLength]; - var key = SpanByte.FromPinnedSpan(keyVec); - - Span inputVec = stackalloc byte[InitialLength]; - var input = SpanByte.FromPinnedSpan(inputVec); + Span key = stackalloc byte[KeyLength]; + Span input = stackalloc byte[InitialLength]; + var pinnedInputSpan = PinnedSpanByte.FromPinnedSpan(input); RevivificationStressFunctions localFunctions = new RevivificationStressFunctions(keyComparer: null); - using var localSession = store.NewSession(localFunctions); + using var localSession = store.NewSession(localFunctions); var localbContext = localSession.BasicContext; for (var iteration = 0; iteration < numIterations; ++iteration) { for (var ii = tid; ii < NumRecords; ii += numUpdateThreads) { - keyVec.Fill((byte)ii); - inputVec.Fill((byte)ii); + key.Fill((byte)ii); + input.Fill((byte)ii); - localSession.functions.expectedKey = key; - _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, input); + localSession.functions.expectedKey = PinnedSpanByte.FromPinnedSpan(key); + _ = updateOp == UpdateOp.Upsert ? localbContext.Upsert(key, input) : localbContext.RMW(key, ref pinnedInputSpan); localSession.functions.expectedKey = default; } diff --git a/libs/storage/Tsavorite/cs/test/SessionTests.cs b/libs/storage/Tsavorite/cs/test/SessionTests.cs index 7cdbbf88d7c..9ed5408e870 100644 --- a/libs/storage/Tsavorite/cs/test/SessionTests.cs +++ b/libs/storage/Tsavorite/cs/test/SessionTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System.IO; using System.Threading.Tasks; using NUnit.Framework; @@ -264,4 +266,6 @@ public void SessionTest5() session.Dispose(); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/SharedDirectoryTests.cs b/libs/storage/Tsavorite/cs/test/SharedDirectoryTests.cs index 8c42910efbf..39c3f627b7d 100644 --- a/libs/storage/Tsavorite/cs/test/SharedDirectoryTests.cs +++ b/libs/storage/Tsavorite/cs/test/SharedDirectoryTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -244,4 +246,6 @@ private static void CopyDirectory(DirectoryInfo source, DirectoryInfo target) } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/SimpleRecoveryTest.cs b/libs/storage/Tsavorite/cs/test/SimpleRecoveryTest.cs index 12791eab5c9..dc2ca06e749 100644 --- a/libs/storage/Tsavorite/cs/test/SimpleRecoveryTest.cs +++ b/libs/storage/Tsavorite/cs/test/SimpleRecoveryTest.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Linq; @@ -430,15 +432,7 @@ public override void ReadCompletionCallback(ref AdId key, ref AdInput input, ref } // Read functions - public override bool SingleReader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo) - { - if (expectedVersion >= 0) - ClassicAssert.AreEqual(expectedVersion, readInfo.Version); - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref AdId key, ref AdInput input, ref NumClicks value, ref Output dst, ref ReadInfo readInfo) { if (expectedVersion >= 0) ClassicAssert.AreEqual(expectedVersion, readInfo.Version); @@ -478,4 +472,6 @@ public override bool CopyUpdater(ref AdId key, ref AdInput input, ref NumClicks return true; } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/SingleWriterTests.cs b/libs/storage/Tsavorite/cs/test/SingleWriterTests.cs deleted file mode 100644 index da42b4a8dc3..00000000000 --- a/libs/storage/Tsavorite/cs/test/SingleWriterTests.cs +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.IO; -using NUnit.Framework; -using NUnit.Framework.Legacy; -using Tsavorite.core; -using static Tsavorite.test.TestUtils; - -namespace Tsavorite.test.SingleWriter -{ - using IntAllocator = BlittableAllocator>>; - using IntStoreFunctions = StoreFunctions>; - - internal class SingleWriterTestFunctions : SimpleSimpleFunctions - { - internal WriteReason actualReason; - - public override bool SingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - { - ClassicAssert.AreEqual((WriteReason)input, reason); - actualReason = reason; - return true; - } - - public override void PostSingleWriter(ref int key, ref int input, ref int src, ref int dst, ref int output, ref UpsertInfo upsertInfo, WriteReason reason) - { - ClassicAssert.AreEqual((WriteReason)input, reason); - actualReason = reason; - } - } - - class SingleWriterTests - { - const int NumRecords = 1000; - const int ValueMult = 1_000_000; - const WriteReason NoReason = (WriteReason)255; - - SingleWriterTestFunctions functions; - - private TsavoriteKV store; - private ClientSession session; - private BasicContext bContext; - private IDevice log; - - [SetUp] - public void Setup() - { - DeleteDirectory(MethodTestDir, wait: true); - log = Devices.CreateLogDevice(Path.Combine(MethodTestDir, "test.log"), deleteOnClose: false); - - functions = new(); - KVSettings kvSettings = new() - { - IndexSize = 1L << 26, - LogDevice = log, - PageSize = 1L << 12, - MemorySize = 1L << 22, - ReadCopyOptions = new(ReadCopyFrom.Device, ReadCopyTo.MainLog), - CheckpointDir = MethodTestDir - }; - foreach (var arg in TestContext.CurrentContext.Test.Arguments) - { - if (arg is ReadCopyDestination dest) - { - if (dest == ReadCopyDestination.ReadCache) - { - kvSettings.ReadCachePageSize = 1L << 12; - kvSettings.ReadCacheMemorySize = 1L << 22; - kvSettings.ReadCacheEnabled = true; - kvSettings.ReadCopyOptions = default; - } - break; - } - } - - store = new(kvSettings - , StoreFunctions.Create(IntKeyComparer.Instance) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - session = store.NewSession(functions); - bContext = session.BasicContext; - } - - [TearDown] - public void TearDown() - { - session?.Dispose(); - session = null; - store?.Dispose(); - store = null; - log?.Dispose(); - log = null; - DeleteDirectory(MethodTestDir); - } - - void Populate() - { - int input = (int)WriteReason.Upsert; - int output = 0; - for (int key = 0; key < NumRecords; key++) - ClassicAssert.False(bContext.Upsert(key, input, key * ValueMult, ref output).IsPending); - } - - [Test] - [Category(TsavoriteKVTestCategory)] - [Category(SmokeTestCategory)] - public void SingleWriterReasonsTest([Values] ReadCopyDestination readCopyDestination) - { - functions.actualReason = NoReason; - Populate(); - ClassicAssert.AreEqual(WriteReason.Upsert, functions.actualReason); - - store.Log.FlushAndEvict(wait: true); - - functions.actualReason = NoReason; - int key = 42; - WriteReason expectedReason = readCopyDestination == ReadCopyDestination.ReadCache ? WriteReason.CopyToReadCache : WriteReason.CopyToTail; - int input = (int)expectedReason; - var status = bContext.Read(key, input, out int output); - ClassicAssert.IsTrue(status.IsPending); - _ = bContext.CompletePending(wait: true); - ClassicAssert.AreEqual(expectedReason, functions.actualReason); - - functions.actualReason = NoReason; - key = 64; - expectedReason = WriteReason.CopyToTail; - input = (int)expectedReason; - ReadOptions readOptions = new() { CopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.MainLog) }; - status = bContext.Read(ref key, ref input, ref output, ref readOptions, out _); - ClassicAssert.IsTrue(status.IsPending && !status.IsCompleted); - _ = bContext.CompletePendingWithOutputs(out var outputs, wait: true); - (status, output) = GetSinglePendingResult(outputs); - ClassicAssert.IsTrue(!status.IsPending && status.IsCompleted && status.IsCompletedSuccessfully); - ClassicAssert.IsTrue(status.Found && !status.NotFound && status.Record.Copied); - ClassicAssert.AreEqual(expectedReason, functions.actualReason); - - functions.actualReason = NoReason; - expectedReason = WriteReason.Compaction; - input = (int)expectedReason; - _ = store.Log.Compact(functions, ref input, ref output, store.Log.SafeReadOnlyAddress, CompactionType.Scan); - ClassicAssert.AreEqual(expectedReason, functions.actualReason); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/SpanByteIterationTests.cs b/libs/storage/Tsavorite/cs/test/SpanByteIterationTests.cs index 9a379b7cf2d..f47b8528671 100644 --- a/libs/storage/Tsavorite/cs/test/SpanByteIterationTests.cs +++ b/libs/storage/Tsavorite/cs/test/SpanByteIterationTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -17,7 +19,7 @@ namespace Tsavorite.test [TestFixture] internal class SpanByteIterationTests { - private TsavoriteKV> store; + private TsavoriteKV> store; private IDevice log; // Note: We always set value.length to 2, which includes both VLValue members; we are not exercising the "Variable Length" aspect here. @@ -46,7 +48,7 @@ internal struct SpanBytePushIterationTestFunctions : IScanIteratorFunctions 0) @@ -58,9 +60,6 @@ public unsafe bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMeta return stopAt != ++numRecords; } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - public readonly bool OnStart(long beginAddress, long endAddress) => true; public readonly void OnException(Exception exception, long numberOfRecords) { } public readonly void OnStop(bool completed, long numberOfRecords) { } @@ -71,16 +70,13 @@ internal struct IterationCollisionTestFunctions : IScanIteratorFunctions keys; public IterationCollisionTestFunctions() => keys = new(); - public unsafe bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public unsafe bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) { keys.Add(*(long*)key.ToPointer()); cursorRecordResult = CursorRecordResult.Accept; // default; not used here return true; } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => SingleReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); - public readonly bool OnStart(long beginAddress, long endAddress) => true; public readonly void OnException(Exception exception, long numberOfRecords) { } public readonly void OnStop(bool completed, long numberOfRecords) { } @@ -119,7 +115,7 @@ void iterateAndVerify(int keyMultToValue, int expectedRecs) { using var iter = session.Iterate(); while (iter.GetNext(out var recordInfo)) - _ = scanIteratorFunctions.SingleReader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); + _ = scanIteratorFunctions.Reader(ref iter.GetKey(), ref iter.GetValue(), default, default, out _); } else ClassicAssert.IsTrue(session.Iterate(ref scanIteratorFunctions), $"Failed to complete push iteration; numRecords = {scanIteratorFunctions.numRecords}"); @@ -320,4 +316,6 @@ void LocalUpdate(int tid) Task.WaitAll([.. tasks]); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/BlittableLogCompactionTests.cs b/libs/storage/Tsavorite/cs/test/SpanByteLogCompactionTests.cs similarity index 64% rename from libs/storage/Tsavorite/cs/test/BlittableLogCompactionTests.cs rename to libs/storage/Tsavorite/cs/test/SpanByteLogCompactionTests.cs index 4d1c2929ea6..3ada15f2b68 100644 --- a/libs/storage/Tsavorite/cs/test/BlittableLogCompactionTests.cs +++ b/libs/storage/Tsavorite/cs/test/SpanByteLogCompactionTests.cs @@ -13,18 +13,18 @@ namespace Tsavorite.test { // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - struct HashModuloKeyStructComparer : IKeyComparer + struct HashModuloKeyStructComparer : IKeyComparer { readonly HashModulo modRange; internal HashModuloKeyStructComparer(HashModulo mod) => modRange = mod; - public readonly bool Equals(ref KeyStruct k1, ref KeyStruct k2) => k1.kfield1 == k2.kfield1; + public readonly bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef().kfield1 == k2.AsRef().kfield1; // Force collisions to create a chain - public readonly long GetHashCode64(ref KeyStruct k) + public readonly long GetHashCode64(ReadOnlySpan k) { - var value = Utility.GetHashCode(k.kfield1); + var value = Utility.GetHashCode(k.AsRef().kfield1); return modRange != HashModulo.NoMod ? value % (long)modRange : value; } } @@ -32,20 +32,20 @@ public readonly long GetHashCode64(ref KeyStruct k) namespace Tsavorite.test { - using StructAllocator = BlittableAllocator>>; - using StructStoreFunctions = StoreFunctions>; + using StructAllocator = SpanByteAllocator>; + using StructStoreFunctions = StoreFunctions; [TestFixture] - public class BlittableLogCompactionTests + public class SpanByteLogCompactionTests { - private TsavoriteKV store; + private TsavoriteKV store; private IDevice log; [SetUp] public void Setup() { DeleteDirectory(MethodTestDir, wait: true); - log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "BlittableLogCompactionTests.log"), deleteOnClose: true); + log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "SpanByteLogCompactionTests.log"), deleteOnClose: true); var hashMod = HashModulo.NoMod; foreach (var arg in TestContext.CurrentContext.Test.Arguments) @@ -63,7 +63,7 @@ public void Setup() LogDevice = log, MemorySize = 1L << 15, PageSize = 1L << 9 - }, StoreFunctions.Create(new HashModuloKeyStructComparer(hashMod)) + }, StoreFunctions.Create(new HashModuloKeyStructComparer(hashMod), SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); } @@ -78,7 +78,7 @@ public void TearDown() DeleteDirectory(MethodTestDir); } - static void VerifyRead(ClientSession session, int totalRecords, Func isDeleted) + static void VerifyRead(ClientSession session, int totalRecords, Func isDeleted) { InputStruct input = default; int numPending = 0; @@ -91,14 +91,14 @@ void drainPending() { for (; outputs.Next(); --numPending) { - if (isDeleted((int)outputs.Current.Key.kfield1)) + if (isDeleted((int)outputs.Current.Key.AsRef().kfield1)) { ClassicAssert.IsFalse(outputs.Current.Status.Found); continue; } ClassicAssert.IsTrue(outputs.Current.Status.Found); - ClassicAssert.AreEqual(outputs.Current.Key.kfield1, outputs.Current.Output.value.vfield1); - ClassicAssert.AreEqual(outputs.Current.Key.kfield2, outputs.Current.Output.value.vfield2); + ClassicAssert.AreEqual(outputs.Current.Key.AsRef().kfield1, outputs.Current.Output.value.vfield1); + ClassicAssert.AreEqual(outputs.Current.Key.AsRef().kfield2, outputs.Current.Output.value.vfield2); } } ClassicAssert.AreEqual(numPending, 0); @@ -110,7 +110,7 @@ void drainPending() var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - var status = bContext.Read(ref key1, ref input, ref output, isDeleted(i) ? 1 : 0); + var status = bContext.Read(SpanByte.FromPinnedVariable(ref key1), ref input, ref output, isDeleted(i) ? 1 : 0); if (!status.IsPending) { if (isDeleted(i)) @@ -135,7 +135,7 @@ void drainPending() [Category("Compaction")] [Category("Smoke")] - public void BlittableLogCompactionTest1([Values] CompactionType compactionType) + public void SpanByteLogCompactionTest1([Values] CompactionType compactionType) { using var session = store.NewSession(new FunctionsCompaction()); var bContext = session.BasicContext; @@ -149,9 +149,10 @@ public void BlittableLogCompactionTest1([Values] CompactionType compactionType) if (i == totalRecords - 1000) compactUntil = store.Log.TailAddress; - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + _ = bContext.Upsert(key, value, 0); } store.Log.FlushAndEvict(wait: true); @@ -161,13 +162,13 @@ public void BlittableLogCompactionTest1([Values] CompactionType compactionType) ClassicAssert.AreEqual(compactUntil, store.Log.BeginAddress); // Read all keys - all should be present - BlittableLogCompactionTests.VerifyRead(session, totalRecords, key => false); + VerifyRead(session, totalRecords, key => false); } [Test] [Category("TsavoriteKV")] [Category("Compaction")] - public void BlittableLogCompactionTest2([Values] CompactionType compactionType, [Values(HashModulo.NoMod, HashModulo.Hundred)] HashModulo hashMod) + public void SpanByteLogCompactionTest2([Values] CompactionType compactionType, [Values(HashModulo.NoMod, HashModulo.Hundred)] HashModulo hashMod) { using var session = store.NewSession(new FunctionsCompaction()); var bContext = session.BasicContext; @@ -181,9 +182,10 @@ public void BlittableLogCompactionTest2([Values] CompactionType compactionType, if (i == totalRecords - 1000) compactUntil = store.Log.TailAddress; - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + _ = bContext.Upsert(key, value, 0); } store.Log.FlushAndEvict(true); @@ -199,9 +201,10 @@ public void BlittableLogCompactionTest2([Values] CompactionType compactionType, // test that the address is < minAddress, so no IO is needed. for (int i = 0; i < totalRecords / 2; i++) { - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + _ = bContext.Upsert(key, value, 0); } compactUntil = session.Compact(compactUntil, compactionType); @@ -209,13 +212,13 @@ public void BlittableLogCompactionTest2([Values] CompactionType compactionType, ClassicAssert.AreEqual(compactUntil, store.Log.BeginAddress); // Read all keys - all should be present - BlittableLogCompactionTests.VerifyRead(session, totalRecords, key => false); + VerifyRead(session, totalRecords, key => false); } [Test] [Category("TsavoriteKV")] [Category("Compaction")] - public void BlittableLogCompactionTest3([Values] CompactionType compactionType) + public void SpanByteLogCompactionTest3([Values] CompactionType compactionType) { using var session = store.NewSession(new FunctionsCompaction()); var bContext = session.BasicContext; @@ -229,15 +232,16 @@ public void BlittableLogCompactionTest3([Values] CompactionType compactionType) if (i == totalRecords / 2) compactUntil = store.Log.TailAddress; - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + _ = bContext.Upsert(key, value, 0); if (i % 8 == 0) { int j = i / 4; - key1 = new KeyStruct { kfield1 = j, kfield2 = j + 1 }; - _ = bContext.Delete(ref key1, 0); + keyStruct = new KeyStruct { kfield1 = j, kfield2 = j + 1 }; + _ = bContext.Delete(key, 0); } } @@ -247,7 +251,7 @@ public void BlittableLogCompactionTest3([Values] CompactionType compactionType) ClassicAssert.AreEqual(compactUntil, store.Log.BeginAddress); // Read all keys - all should be present except those we deleted - BlittableLogCompactionTests.VerifyRead(session, totalRecords, key => (key < totalRecords / 4) && (key % 2 == 0)); + VerifyRead(session, totalRecords, key => (key < totalRecords / 4) && (key % 2 == 0)); } [Test] @@ -255,7 +259,7 @@ public void BlittableLogCompactionTest3([Values] CompactionType compactionType) [Category("Compaction")] [Category("Smoke")] - public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType compactionType) + public void SpanByteLogCompactionCustomFunctionsTest1([Values] CompactionType compactionType) { using var session = store.NewSession(new FunctionsCompaction()); var bContext = session.BasicContext; @@ -271,9 +275,10 @@ public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType c if (i == totalRecords / 2) compactUntil = store.Log.TailAddress; - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - _ = bContext.Upsert(ref key1, ref value, 0); + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + _ = bContext.Upsert(key, value, 0); } var tail = store.Log.TailAddress; @@ -287,12 +292,13 @@ public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType c for (var i = 0; i < totalRecords; i++) { OutputStruct output = default; - var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; - var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + var keyStruct = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var valueStruct = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); var ctx = (i < (totalRecords / 2) && (i % 2 != 0)) ? 1 : 0; - var status = bContext.Read(ref key1, ref input, ref output, ctx); + var status = bContext.Read(key, ref input, ref output, ctx); if (status.IsPending) { ClassicAssert.IsTrue(bContext.CompletePendingWithOutputs(out var outputs, wait: true)); @@ -302,8 +308,8 @@ public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType c if (ctx == 0) { ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); - ClassicAssert.AreEqual(value.vfield2, output.value.vfield2); + ClassicAssert.AreEqual(valueStruct.vfield1, output.value.vfield1); + ClassicAssert.AreEqual(valueStruct.vfield2, output.value.vfield2); } else { @@ -315,7 +321,7 @@ public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType c [Test] [Category("TsavoriteKV")] [Category("Compaction")] - public void BlittableLogCompactionCustomFunctionsTest2([Values] CompactionType compactionType, [Values] bool flushAndEvict) + public void SpanByteLogCompactionCustomFunctionsTest2([Values] CompactionType compactionType, [Values] bool flushAndEvict) { // Update: irrelevant as session compaction no longer uses Copy/CopyInPlace // This test checks if CopyInPlace returning false triggers call to Copy @@ -323,20 +329,22 @@ public void BlittableLogCompactionCustomFunctionsTest2([Values] CompactionType c using var session = store.NewSession(new FunctionsCompaction()); var bContext = session.BasicContext; - var key = new KeyStruct { kfield1 = 100, kfield2 = 101 }; - var value = new ValueStruct { vfield1 = 10, vfield2 = 20 }; + var keyStruct = new KeyStruct { kfield1 = 100, kfield2 = 101 }; + var valueStruct = new ValueStruct { vfield1 = 10, vfield2 = 20 }; + ReadOnlySpan key = SpanByte.FromPinnedVariable(ref keyStruct), value = SpanByte.FromPinnedVariable(ref valueStruct); + var input = default(InputStruct); var output = default(OutputStruct); - _ = bContext.Upsert(ref key, ref value, 0); - var status = bContext.Read(ref key, ref input, ref output, 0); + _ = bContext.Upsert(key, value, 0); + var status = bContext.Read(key, ref input, ref output, 0); Debug.Assert(status.Found); store.Log.Flush(true); - value = new ValueStruct { vfield1 = 11, vfield2 = 21 }; - _ = bContext.Upsert(ref key, ref value, 0); - status = bContext.Read(ref key, ref input, ref output, 0); + valueStruct = new ValueStruct { vfield1 = 11, vfield2 = 21 }; + _ = bContext.Upsert(key, value, 0); + status = bContext.Read(key, ref input, ref output, 0); Debug.Assert(status.Found); if (flushAndEvict) @@ -347,7 +355,7 @@ public void BlittableLogCompactionCustomFunctionsTest2([Values] CompactionType c var compactUntil = session.Compact(store.Log.TailAddress, compactionType); store.Log.Truncate(); - status = bContext.Read(ref key, ref input, ref output, 0); + status = bContext.Read(key, ref input, ref output, 0); if (status.IsPending) { ClassicAssert.IsTrue(bContext.CompletePendingWithOutputs(out var outputs, wait: true)); @@ -355,13 +363,15 @@ public void BlittableLogCompactionCustomFunctionsTest2([Values] CompactionType c } ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(value.vfield1, output.value.vfield1); - ClassicAssert.AreEqual(value.vfield2, output.value.vfield2); + ClassicAssert.AreEqual(valueStruct.vfield1, output.value.vfield1); + ClassicAssert.AreEqual(valueStruct.vfield2, output.value.vfield2); } - private struct EvenCompactionFunctions : ICompactionFunctions + private struct EvenCompactionFunctions : ICompactionFunctions { - public readonly bool IsDeleted(ref KeyStruct key, ref ValueStruct value) => value.vfield1 % 2 != 0; + public bool IsDeleted(in TSourceLogRecord logRecord) + where TSourceLogRecord : ISourceLogRecord + => logRecord.ValueSpan.AsRef().vfield1 % 2 != 0; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/SpanByteLogScanTests.cs b/libs/storage/Tsavorite/cs/test/SpanByteLogScanTests.cs index 788523d7953..c407586d75b 100644 --- a/libs/storage/Tsavorite/cs/test/SpanByteLogScanTests.cs +++ b/libs/storage/Tsavorite/cs/test/SpanByteLogScanTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using System.Runtime.InteropServices; @@ -39,7 +41,7 @@ namespace Tsavorite.test.spanbyte [TestFixture] internal class SpanByteLogScanTests { - private TsavoriteKV> store; + private TsavoriteKV> store; private IDevice log; const int TotalRecords = 2000; const int PageSizeBits = 15; @@ -86,10 +88,10 @@ public class ScanFunctions : SpanByteFunctions // Right now this is unused but helped with debugging so I'm keeping it around. internal long insertedAddress; - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref SpanByteAndMemory output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { insertedAddress = upsertInfo.Address; - return base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + return base.InitialWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); } } @@ -285,7 +287,7 @@ public unsafe void SpanByteScanCursorWithRCUTest([Values(RCULocation.RCUBefore, internal sealed class ScanCursorFuncs : IScanIteratorFunctions { - readonly TsavoriteKV> store; + readonly TsavoriteKV> store; internal int numRecords; internal long lastAddress; @@ -295,7 +297,7 @@ internal sealed class ScanCursorFuncs : IScanIteratorFunctions filter; - internal ScanCursorFuncs(TsavoriteKV> store) + internal ScanCursorFuncs(TsavoriteKV> store) { this.store = store; Initialize(verifyKeys: true); @@ -343,7 +345,7 @@ unsafe void CheckForRCU() } } - public bool ConcurrentReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) + public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) { var keyString = new string(MemoryMarshal.Cast(key.AsReadOnlySpan())); var kfield1 = int.Parse(keyString.Substring(keyString.IndexOf('_') + 1)); @@ -375,9 +377,6 @@ public void OnException(Exception exception, long numberOfRecords) public bool OnStart(long beginAddress, long endAddress) => true; public void OnStop(bool completed, long numberOfRecords) { } - - public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult) - => ConcurrentReader(ref key, ref value, recordMetadata, numberOfRecords, out cursorRecordResult); } [Test] @@ -480,4 +479,6 @@ public void SpanByteIterationPendingCollisionTest([Values(HashModulo.Hundred)] H ClassicAssert.AreEqual(totalRecords, scanIteratorFunctions.keys.Count); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/SpanByteTests.cs b/libs/storage/Tsavorite/cs/test/SpanByteTests.cs index 2d5ed4fab7d..f15352ff082 100644 --- a/libs/storage/Tsavorite/cs/test/SpanByteTests.cs +++ b/libs/storage/Tsavorite/cs/test/SpanByteTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Collections.Generic; using System.IO; @@ -29,7 +31,7 @@ public unsafe void SpanByteTest1() try { using var log = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "hlog1.log"), deleteOnClose: true); - using var store = new TsavoriteKV>( + using var store = new TsavoriteKV>( new() { IndexSize = 1L << 13, @@ -93,7 +95,7 @@ public unsafe void MultiRead_SpanByte_Test() try { using var log = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "test.log"), deleteOnClose: true); - using var store = new TsavoriteKV>( + using var store = new TsavoriteKV>( new() { IndexSize = 1L << 16, @@ -216,7 +218,7 @@ public unsafe void ShouldSkipEmptySpaceAtEndOfPage() TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); using var log = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "vl-iter.log"), deleteOnClose: true); - using var store = new TsavoriteKV>( + using var store = new TsavoriteKV>( new() { IndexSize = 1L << 13, @@ -281,4 +283,6 @@ void Set(ref Span keySpan, long keyValue, ref Span valueSpan, int va } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/SpanByteVLVectorTests.cs b/libs/storage/Tsavorite/cs/test/SpanByteVLVectorTests.cs index 49b5783548e..b2694d29a6e 100644 --- a/libs/storage/Tsavorite/cs/test/SpanByteVLVectorTests.cs +++ b/libs/storage/Tsavorite/cs/test/SpanByteVLVectorTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.IO; using NUnit.Framework; @@ -27,7 +29,7 @@ public unsafe void VLVectorSingleKeyTest() DeleteDirectory(MethodTestDir, wait: true); var log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "hlog1.log"), deleteOnClose: true); - var store = new TsavoriteKV>( + var store = new TsavoriteKV>( new() { IndexSize = 1L << 13, @@ -94,7 +96,7 @@ public unsafe void VLVectorMultiKeyTest() DeleteDirectory(MethodTestDir, wait: true); var log = Devices.CreateLogDevice(Path.Join(MethodTestDir, "hlog1.log"), deleteOnClose: true); - var store = new TsavoriteKV>( + var store = new TsavoriteKV>( new() { IndexSize = 1L << 13, @@ -158,4 +160,6 @@ public unsafe void VLVectorMultiKeyTest() DeleteDirectory(MethodTestDir); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs index 55f337019f5..991b2c14f8b 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs @@ -12,8 +12,8 @@ namespace Tsavorite.test.recovery { using static Tsavorite.test.TestUtils; - using LongAllocator = BlittableAllocator>>; - using LongStoreFunctions = StoreFunctions>; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; public abstract class StateMachineDriverTestsBase { @@ -42,19 +42,19 @@ protected void BaseTearDown() DeleteDirectory(MethodTestDir, true); } - protected abstract void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store); + protected abstract void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store); public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType checkpointType, long indexSize, bool useTimingFuzzing) { // Create the original store - using var store1 = new TsavoriteKV(new() + using var store1 = new TsavoriteKV(new() { IndexSize = indexSize, LogDevice = log, PageSize = 1L << 10, MemorySize = 1L << 20, CheckpointDir = MethodTestDir - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); @@ -92,7 +92,7 @@ public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType for (long key = 0; key < numKeys; key++) { long output = default; - var status = bc1.Read(ref key, ref output); + var status = bc1.Read(SpanByte.FromPinnedVariable(ref key), ref output); if (status.IsPending) { var completed = bc1.CompletePendingWithOutputs(out var completedOutputs, true); @@ -111,7 +111,7 @@ public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType } // Recover new store from the checkpoint - using var store2 = new TsavoriteKV(new() + using var store2 = new TsavoriteKV(new() { IndexSize = indexSize, LogDevice = log, @@ -119,7 +119,7 @@ public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType PageSize = 1L << 10, MemorySize = 1L << 20, CheckpointDir = MethodTestDir - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); _ = await store2.RecoverAsync(default, checkpointToken); @@ -130,7 +130,7 @@ public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType for (long key = 0; key < numKeys; key++) { long output = default; - var status = bc2.Read(ref key, ref output); + var status = bc2.Read(SpanByte.FromPinnedVariable(ref key), ref output); if (status.IsPending) { var completed = bc2.CompletePendingWithOutputs(out var completedOutputs, true); @@ -159,14 +159,14 @@ public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck(CheckpointType public async ValueTask DoGrowIndexVersionSwitchEquivalenceCheck(long indexSize, bool useTimingFuzzing) { // Create the original store - using var store1 = new TsavoriteKV(new() + using var store1 = new TsavoriteKV(new() { IndexSize = indexSize, LogDevice = log, PageSize = 1L << 10, MemorySize = 1L << 20, CheckpointDir = MethodTestDir - }, StoreFunctions.Create(LongKeyComparer.Instance) + }, StoreFunctions.Create(LongKeyComparer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); @@ -202,7 +202,7 @@ public async ValueTask DoGrowIndexVersionSwitchEquivalenceCheck(long indexSize, for (long key = 0; key < numKeys; key++) { long output = default; - var status = bc1.Read(ref key, ref output); + var status = bc1.Read(SpanByte.FromPinnedVariable(ref key), ref output); if (status.IsPending) { var completed = bc1.CompletePendingWithOutputs(out var completedOutputs, true); @@ -222,7 +222,7 @@ public async ValueTask DoGrowIndexVersionSwitchEquivalenceCheck(long indexSize, } } - public class SumFunctions : SimpleSimpleFunctions + public class SumFunctions : SimpleLongSimpleFunctions { readonly Random fuzzer; @@ -231,18 +231,18 @@ public SumFunctions(int thread_id, bool useTimingFuzzing) : base((l, r) => l + r if (useTimingFuzzing) fuzzer = new Random(thread_id); } - public override bool InPlaceUpdater(ref long key, ref long input, ref long value, ref long output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { Fuzz(); - var ret = base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); + var ret = base.InPlaceUpdater(ref logRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); Fuzz(); return ret; } - public override bool CopyUpdater(ref long key, ref long input, ref long oldValue, ref long newValue, ref long output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { Fuzz(); - var ret = base.CopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo, ref recordInfo); + var ret = base.CopyUpdater(ref srcLogRecord, ref dstLogRecord, ref sizeInfo, ref input, ref output, ref rmwInfo); Fuzz(); return ret; } @@ -263,7 +263,7 @@ public class CheckpointVersionSwitchRmw : StateMachineDriverTestsBase [TearDown] public void TearDown() => BaseTearDown(); - protected override void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store) + protected override void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store) { using var s = store.NewSession(new SumFunctions(thread_id, useTimingFuzzing)); var bc = s.BasicContext; @@ -279,7 +279,7 @@ protected override void OperationThread(int thread_id, bool useTimingFuzzing, Ts key = r.Next(numKeys); // Run the RMW operation - _ = bc.RMW(ref key, ref input); + _ = bc.RMW(SpanByte.FromPinnedVariable(ref key), ref input); // Update expected counts for the old and new version of store if (bc.Session.Version == currentIteration + 1) @@ -318,10 +318,10 @@ public class CheckpointVersionSwitchTxn : StateMachineDriverTestsBase [TearDown] public void TearDown() => BaseTearDown(); - protected override void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store) + protected override void OperationThread(int thread_id, bool useTimingFuzzing, TsavoriteKV store) { using var s = store.NewSession(new SumFunctions(thread_id, useTimingFuzzing)); - var lc = s.LockableContext; + var lc = s.TransactionalContext; var r = new Random(thread_id); ClassicAssert.IsTrue(numKeys > 1); @@ -338,15 +338,15 @@ protected override void OperationThread(int thread_id, bool useTimingFuzzing, Ts key2 = r.Next(numKeys); } while (key2 == key1); - var exclusiveVec = new FixedLengthLockableKeyStruct[] { - new(key1, LockType.Exclusive, lc), - new(key2, LockType.Exclusive, lc) + var exclusiveVec = new FixedLengthTransactionalKeyStruct[] { + new(SpanByte.FromPinnedVariable(ref key1), LockType.Exclusive, lc), + new(SpanByte.FromPinnedVariable(ref key2), LockType.Exclusive, lc) }; var txnVersion = store.stateMachineDriver.AcquireTransactionVersion(); // Start transaction, session does not acquire version in this call - lc.BeginLockable(); + lc.BeginTransaction(); // Lock keys, session acquires version in this call lc.Lock(exclusiveVec); @@ -355,14 +355,14 @@ protected override void OperationThread(int thread_id, bool useTimingFuzzing, Ts lc.LocksAcquired(txnVersion); // Run transaction - _ = lc.RMW(ref key1, ref input); - _ = lc.RMW(ref key2, ref input); + _ = lc.RMW(SpanByte.FromPinnedVariable(ref key1), ref input); + _ = lc.RMW(SpanByte.FromPinnedVariable(ref key2), ref input); // Unlock keys lc.Unlock(exclusiveVec); // End transaction - lc.EndLockable(); + lc.EndTransaction(); store.stateMachineDriver.EndTransaction(txnVersion); diff --git a/libs/storage/Tsavorite/cs/test/StructWithStringTests.cs b/libs/storage/Tsavorite/cs/test/StructWithStringTests.cs deleted file mode 100644 index 13cbde8f92c..00000000000 --- a/libs/storage/Tsavorite/cs/test/StructWithStringTests.cs +++ /dev/null @@ -1,164 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.IO; -using NUnit.Framework; -using NUnit.Framework.Legacy; -using Tsavorite.core; -using static Tsavorite.test.TestUtils; - -namespace Tsavorite.test.StructWithString -{ - // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - public struct StructWithString(int intValue, string prefix) - { - public int intField = intValue; - public string stringField = prefix + intValue.ToString(); - - public override readonly string ToString() => stringField; - - public class Comparer : IKeyComparer - { - public long GetHashCode64(ref StructWithString k) => Utility.GetHashCode(k.intField); - - public bool Equals(ref StructWithString k1, ref StructWithString k2) => k1.intField == k2.intField && k1.stringField == k2.stringField; - } - - public class Serializer : BinaryObjectSerializer - { - public override void Deserialize(out StructWithString obj) - { - var intField = reader.ReadInt32(); - var stringField = reader.ReadString(); - obj = new() { intField = intField, stringField = stringField }; - } - - public override void Serialize(ref StructWithString obj) - { - writer.Write(obj.intField); - writer.Write(obj.stringField); - } - } - } -} - -namespace Tsavorite.test.StructWithString -{ - using ClassAllocator = GenericAllocator>>; - using ClassStoreFunctions = StoreFunctions>; - - [TestFixture] - public class StructWithStringTests - { - internal class StructWithStringTestFunctions : SimpleSimpleFunctions - { - } - - const int NumRecords = 1_000; - const string KeyPrefix = "key_"; - string valuePrefix = "value_"; - - StructWithStringTestFunctions functions; - - private TsavoriteKV store; - private ClientSession session; - private BasicContext bContext; - private IDevice log, objlog; - - [SetUp] - public void Setup() - { - // create a string of size 1024 bytes - valuePrefix = new string('a', 1024); - - DeleteDirectory(MethodTestDir, wait: true); - log = Devices.CreateLogDevice(Path.Combine(MethodTestDir, "test.log"), deleteOnClose: false); - objlog = Devices.CreateLogDevice(Path.Combine(MethodTestDir, "test.obj.log"), deleteOnClose: false); - - store = new(new() - { - IndexSize = 1L << 26, - LogDevice = log, - ObjectLogDevice = objlog, - PageSize = 1L << 10, - MemorySize = 1L << 22, - SegmentSize = 1L << 16, - CheckpointDir = MethodTestDir - }, StoreFunctions.Create(new StructWithString.Comparer(), () => new StructWithString.Serializer(), () => new StructWithString.Serializer()) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - - functions = new(); - session = store.NewSession(functions); - bContext = session.BasicContext; - } - - [TearDown] - public void TearDown() - { - session?.Dispose(); - session = null; - store?.Dispose(); - store = null; - objlog?.Dispose(); - objlog = null; - log?.Dispose(); - log = null; - DeleteDirectory(MethodTestDir); - } - - void Populate() - { - for (int ii = 0; ii < NumRecords; ii++) - { - StructWithString key = new(ii, KeyPrefix); - StructWithString value = new(ii, valuePrefix); - bContext.Upsert(ref key, ref value); - if (ii % 3_000 == 0) - { - store.TakeHybridLogCheckpointAsync(CheckpointType.FoldOver).GetAwaiter().GetResult(); - store.Recover(); - } - } - } - - [Test] - [Category(TsavoriteKVTestCategory)] - [Category(SmokeTestCategory)] - public void StructWithStringCompactTest([Values] CompactionType compactionType, [Values] bool flush) - { - void readKey(int keyInt) - { - StructWithString key = new(keyInt, KeyPrefix); - var (status, output) = bContext.Read(key); - bool wasPending = status.IsPending; - if (status.IsPending) - { - bContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); - using (completedOutputs) - (status, output) = GetSinglePendingResult(completedOutputs); - } - - ClassicAssert.IsTrue(status.Found, $"{status.ToString()}; wasPending = {wasPending}"); - ClassicAssert.AreEqual(key.intField, output.intField); - } - - Populate(); - readKey(12); - if (flush) - { - store.Log.FlushAndEvict(wait: true); - readKey(24); - } - int count = 0; - using var iter = store.Log.Scan(0, store.Log.TailAddress); - while (iter.GetNext(out var _)) - count++; - ClassicAssert.AreEqual(count, NumRecords); - - readKey(48); - store.Log.Compact(functions, store.Log.SafeReadOnlyAddress, compactionType); - readKey(48); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/TestTypes.cs b/libs/storage/Tsavorite/cs/test/TestTypes.cs index 4d63a059f34..9883e5d83b4 100644 --- a/libs/storage/Tsavorite/cs/test/TestTypes.cs +++ b/libs/storage/Tsavorite/cs/test/TestTypes.cs @@ -2,40 +2,58 @@ // Licensed under the MIT license. using System; +using System.Runtime.InteropServices; using System.Threading; using NUnit.Framework.Legacy; using Tsavorite.core; +#pragma warning disable CA2211 // Non-constant fields should not be visible (This is for the .Instance members) + namespace Tsavorite.test { - public struct KeyStruct + [StructLayout(LayoutKind.Explicit)] + public unsafe struct KeyStruct { + [FieldOffset(0)] public long kfield1; + [FieldOffset(8)] public long kfield2; - public override string ToString() => $"kfield1 {kfield1}, kfield2 {kfield2}"; + public override readonly string ToString() => $"kfield1 {kfield1}, kfield2 {kfield2}"; - public struct Comparer : IKeyComparer + public struct Comparer : IKeyComparer { - public long GetHashCode64(ref KeyStruct key) => Utility.GetHashCode(key.kfield1); - public bool Equals(ref KeyStruct k1, ref KeyStruct k2) => k1.kfield1 == k2.kfield1 && k1.kfield2 == k2.kfield2; + public readonly long GetHashCode64(ReadOnlySpan key) => Utility.GetHashCode(key.AsRef().kfield1); + + public readonly bool Equals(ReadOnlySpan key1, ReadOnlySpan key2) + { + var k1 = key1.AsRef(); + var k2 = key2.AsRef(); + return k1.kfield1 == k2.kfield1 && k1.kfield2 == k2.kfield2; + } public static Comparer Instance = new(); } } - public struct ValueStruct + [StructLayout(LayoutKind.Explicit)] + public unsafe struct ValueStruct { + [FieldOffset(0)] public long vfield1; + [FieldOffset(8)] public long vfield2; - public override string ToString() => $"vfield1 {vfield1}, vfield2 {vfield2}"; + + public static int AsSpanByteDataSize => sizeof(ValueStruct); + + public override readonly string ToString() => $"vfield1 {vfield1}, vfield2 {vfield2}"; } public struct InputStruct { public long ifield1; public long ifield2; - public override string ToString() => $"ifield1 {ifield1}, ifield2 {ifield2}"; + public override readonly string ToString() => $"ifield1 {ifield1}, ifield2 {ifield2}"; } public struct OutputStruct @@ -47,93 +65,98 @@ public struct ContextStruct { public long cfield1; public long cfield2; - public override string ToString() => $"cfield1 {cfield1}, cfield2 {cfield2}"; + public override readonly string ToString() => $"cfield1 {cfield1}, cfield2 {cfield2}"; } public class Functions : FunctionsWithContext { } - public class FunctionsWithContext : SessionFunctionsBase + public class FunctionsWithContext : SessionFunctionsBase { - public override void RMWCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, TContext ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, TContext ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); ClassicAssert.IsTrue(status.Record.CopyUpdated); - ClassicAssert.AreEqual(key.kfield1 + input.ifield1, output.value.vfield1); - ClassicAssert.AreEqual(key.kfield2 + input.ifield2, output.value.vfield2); + ClassicAssert.AreEqual(diskLogRecord.Key.AsRef().kfield1 + input.ifield1, output.value.vfield1); + ClassicAssert.AreEqual(diskLogRecord.Key.AsRef().kfield2 + input.ifield2, output.value.vfield2); } - public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, TContext ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, TContext ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(key.kfield1, output.value.vfield1); - ClassicAssert.AreEqual(key.kfield2, output.value.vfield2); + ClassicAssert.AreEqual(diskLogRecord.Key.AsRef().kfield1, output.value.vfield1); + ClassicAssert.AreEqual(diskLogRecord.Key.AsRef().kfield2, output.value.vfield2); } // Read functions - public override bool SingleReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref InputStruct input, ref OutputStruct output, ref ReadInfo readInfo) { - ClassicAssert.IsFalse(readInfo.RecordInfo.IsNull()); - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) - { - ClassicAssert.IsFalse(readInfo.RecordInfo.IsNull()); - dst.value = value; + output.value = srcLogRecord.ValueSpan.AsRef(); return true; } // RMW functions - public override bool InitialUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + ref var value = ref logRecord.ValueSpan.AsRef(); value.vfield1 = input.ifield1; value.vfield2 = input.ifield2; output.value = value; return true; } - public override bool InPlaceUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + ref var value = ref logRecord.ValueSpan.AsRef(); value.vfield1 += input.ifield1; value.vfield2 += input.ifield2; output.value = value; return true; } - public override bool NeedCopyUpdate(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref OutputStruct output, ref RMWInfo rmwInfo) + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + ClassicAssert.IsTrue(srcLogRecord.IsSet); return true; } - public override bool CopyUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref ValueStruct newValue, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + var oldValue = srcLogRecord.ValueSpan.AsRef(); + ref var newValue = ref dstLogRecord.ValueSpan.AsRef(); + newValue.vfield1 = oldValue.vfield1 + input.ifield1; newValue.vfield2 = oldValue.vfield2 + input.ifield2; output.value = newValue; return true; } + + /// + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref InputStruct input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; } - public class FunctionsCompaction : SessionFunctionsBase + public class FunctionsCompaction : SessionFunctionsBase { - public override void RMWCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, int ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, int ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); ClassicAssert.IsTrue(status.Record.CopyUpdated); } - public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, int ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, int ctx, Status status, RecordMetadata recordMetadata) { if (ctx == 0) { ClassicAssert.IsTrue(status.Found); + var key = diskLogRecord.Key.AsRef(); ClassicAssert.AreEqual(key.kfield1, output.value.vfield1); ClassicAssert.AreEqual(key.kfield2, output.value.vfield2); } @@ -144,150 +167,199 @@ public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct i } // Read functions - public override bool SingleReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) - { - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref InputStruct input, ref OutputStruct output, ref ReadInfo readInfo) { - dst.value = value; + output.value = srcLogRecord.ValueSpan.AsRef(); return true; } // RMW functions - public override bool InitialUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { + ref var value = ref logRecord.ValueSpan.AsRef(); value.vfield1 = input.ifield1; value.vfield2 = input.ifield2; return true; } - public override bool InPlaceUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { + ref var value = ref logRecord.ValueSpan.AsRef(); value.vfield1 += input.ifield1; value.vfield2 += input.ifield2; return true; } - public override bool NeedCopyUpdate(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref OutputStruct output, ref RMWInfo rmwInfo) => true; + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) => true; - public override bool CopyUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref ValueStruct newValue, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { + var oldValue = srcLogRecord.ValueSpan.AsRef(); + ref var newValue = ref dstLogRecord.ValueSpan.AsRef(); + newValue.vfield1 = oldValue.vfield1 + input.ifield1; newValue.vfield2 = oldValue.vfield2 + input.ifield2; return true; } + + /// + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref InputStruct input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; } - public class FunctionsCopyOnWrite : SessionFunctionsBase + public class FunctionsCopyOnWrite : SessionFunctionsBase { - private int _concurrentWriterCallCount; - private int _inPlaceUpdaterCallCount; + private int inPlaceWriterCallCount; + private int inPlaceUpdaterCallCount; - public int ConcurrentWriterCallCount => _concurrentWriterCallCount; - public int InPlaceUpdaterCallCount => _inPlaceUpdaterCallCount; + public int InPlaceWriterCallCount => inPlaceWriterCallCount; + public int InPlaceUpdaterCallCount => inPlaceUpdaterCallCount; - public override void RMWCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void RMWCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); ClassicAssert.IsTrue(status.Record.CopyUpdated); } - public override void ReadCompletionCallback(ref KeyStruct key, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) + public override void ReadCompletionCallback(ref DiskLogRecord diskLogRecord, ref InputStruct input, ref OutputStruct output, Empty ctx, Status status, RecordMetadata recordMetadata) { ClassicAssert.IsTrue(status.Found); + var key = diskLogRecord.Key.AsRef(); ClassicAssert.AreEqual(key.kfield1, output.value.vfield1); ClassicAssert.AreEqual(key.kfield2, output.value.vfield2); } // Read functions - public override bool SingleReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo) - { - ClassicAssert.IsFalse(readInfo.RecordInfo.IsNull()); - dst.value = value; - return true; - } - - public override bool ConcurrentReader(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref TSourceLogRecord logRecord, ref InputStruct input, ref OutputStruct output, ref ReadInfo readInfo) { - ClassicAssert.IsFalse(readInfo.RecordInfo.IsNull()); - dst.value = value; + output.value = logRecord.ValueSpan.AsRef(); return true; } // Upsert functions - public override bool SingleWriter(ref KeyStruct key, ref InputStruct input, ref ValueStruct src, ref ValueStruct dst, ref OutputStruct output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) + public override bool InitialWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ReadOnlySpan srcValue, ref OutputStruct output, ref UpsertInfo upsertInfo) { - ClassicAssert.IsFalse(upsertInfo.RecordInfo.IsNull()); - dst = src; + logRecord.ValueSpan.AsRef() = srcValue.AsRef(); return true; } - public override bool ConcurrentWriter(ref KeyStruct key, ref InputStruct input, ref ValueStruct src, ref ValueStruct dst, ref OutputStruct output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ReadOnlySpan srcValue, ref OutputStruct output, ref UpsertInfo upsertInfo) { - ClassicAssert.IsFalse(upsertInfo.RecordInfo.IsNull()); - Interlocked.Increment(ref _concurrentWriterCallCount); + _ = Interlocked.Increment(ref inPlaceWriterCallCount); return false; } // RMW functions - public override bool InitialUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + ref var value = ref logRecord.ValueSpan.AsRef(); value.vfield1 = input.ifield1; value.vfield2 = input.ifield2; return true; } - public override bool InPlaceUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct value, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); - Interlocked.Increment(ref _inPlaceUpdaterCallCount); + _ = Interlocked.Increment(ref inPlaceUpdaterCallCount); return false; } - public override bool NeedCopyUpdate(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref OutputStruct output, ref RMWInfo rmwInfo) + public override bool NeedCopyUpdate(ref TSourceLogRecord srcLogRecord, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + ClassicAssert.IsTrue(srcLogRecord.IsSet); return true; } - public override bool CopyUpdater(ref KeyStruct key, ref InputStruct input, ref ValueStruct oldValue, ref ValueStruct newValue, ref OutputStruct output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref InputStruct input, ref OutputStruct output, ref RMWInfo rmwInfo) { - ClassicAssert.IsFalse(rmwInfo.RecordInfo.IsNull()); + var oldValue = srcLogRecord.ValueSpan.AsRef(); + ref var newValue = ref dstLogRecord.ValueSpan.AsRef(); + newValue.vfield1 = oldValue.vfield1 + input.ifield1; newValue.vfield2 = oldValue.vfield2 + input.ifield2; return true; } + + /// + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref InputStruct input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(ValueStruct) }; + /// + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref InputStruct input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; } - class RMWSimpleFunctions : SimpleSimpleFunctions + public class SimpleLongSimpleFunctions : SessionFunctionsBase { - public RMWSimpleFunctions(Func merger) : base(merger) { } + private readonly Func merger; + + public SimpleLongSimpleFunctions() : base() => merger = (input, oldValue) => input; - public override bool InitialUpdater(ref TKey key, ref TValue input, ref TValue value, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public SimpleLongSimpleFunctions(Func merger) => this.merger = merger; + + /// + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref long input, ref long output, ref ReadInfo readInfo) { - base.InitialUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); - output = input; + output = srcLogRecord.ValueSpan.AsRef(); return true; } + public override bool InitialWriter(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ReadOnlySpan srcValue, ref long output, ref UpsertInfo upsertInfo) + { + var result = base.InitialWriter(ref dstLogRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + if (result) + output = srcValue.AsRef(); + return result; + } + + public override bool InPlaceWriter(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref long input, ReadOnlySpan srcValue, ref long output, ref UpsertInfo upsertInfo) + { + var result = base.InPlaceWriter(ref logRecord, ref sizeInfo, ref input, srcValue, ref output, ref upsertInfo); + if (result) + output = srcValue.AsRef(); + return result; + } + /// - public override bool CopyUpdater(ref TKey key, ref TValue input, ref TValue oldValue, ref TValue newValue, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool InitialUpdater(ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { - base.CopyUpdater(ref key, ref input, ref oldValue, ref newValue, ref output, ref rmwInfo, ref recordInfo); - output = newValue; - return true; + var ok = dstLogRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref input), ref sizeInfo); + if (ok) + output = input; + return ok; } /// - public override bool InPlaceUpdater(ref TKey key, ref TValue input, ref TValue value, ref TValue output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) { - base.InPlaceUpdater(ref key, ref input, ref value, ref output, ref rmwInfo, ref recordInfo); - output = value; - return true; + ClassicAssert.IsTrue(dstLogRecord.TryCopyFrom(ref srcLogRecord, ref sizeInfo), "Failed TryCopyRecordValues"); + var result = output = merger(input, srcLogRecord.ValueSpan.AsRef()); // 'result' must be local for SpanByte.From; 'output' may be on the heap + return dstLogRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref result), ref sizeInfo); } + + /// + public override bool InPlaceUpdater(ref LogRecord logRecord, ref RecordSizeInfo sizeInfo, ref long input, ref long output, ref RMWInfo rmwInfo) + { + var result = output = merger(input, logRecord.ValueSpan.AsRef()); // 'result' must be local for SpanByte.From; 'output' may be on the heap + return logRecord.TrySetValueSpan(SpanByte.FromPinnedVariable(ref result), ref sizeInfo); + } + + /// + public override RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref long input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = sizeof(long) }; + /// + public override RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref long input) + => new() { KeyDataSize = key.Length, ValueDataSize = sizeof(long) }; + /// + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, ReadOnlySpan value, ref long input) + => new() { KeyDataSize = key.Length, ValueDataSize = value.Length }; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/TestUtils.cs b/libs/storage/Tsavorite/cs/test/TestUtils.cs index 320919d111e..65a2848fbe2 100644 --- a/libs/storage/Tsavorite/cs/test/TestUtils.cs +++ b/libs/storage/Tsavorite/cs/test/TestUtils.cs @@ -2,7 +2,10 @@ // Licensed under the MIT license. using System; +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; @@ -21,7 +24,7 @@ public static class TestUtils internal const string StressTestCategory = "Stress"; internal const string TsavoriteKVTestCategory = "TsavoriteKV"; internal const string ReadTestCategory = "Read"; - internal const string LockableUnsafeContextTestCategory = "LockableUnsafeContext"; + internal const string TransactionalUnsafeContextTestCategory = "TransactionalUnsafeContext"; internal const string ReadCacheTestCategory = "ReadCache"; internal const string LockTestCategory = "Locking"; internal const string LockTableTestCategory = "LockTable"; @@ -31,6 +34,10 @@ public static class TestUtils internal const string IteratorCategory = "Iterator"; internal const string ModifiedBitTestCategory = "ModifiedBitTest"; internal const string RevivificationCategory = "Revivification"; + internal const string MultiLevelPageArrayCategory = "MultiLevelPageArray"; + internal const string ObjectIdMapCategory = "ObjectIdMap"; + internal const string OverflowFieldCategory = "OverflowField"; + internal const string LogRecordCategory = "LogRecord"; public static ILoggerFactory TestLoggerFactory = CreateLoggerFactoryInstance(TestContext.Progress, LogLevel.Trace); @@ -231,10 +238,10 @@ public enum ScanMode { Scan, Iterate } public enum WaitMode { Wait, NoWait } - internal static (Status status, TOutput output) GetSinglePendingResult(CompletedOutputIterator completedOutputs) + internal static (Status status, TOutput output) GetSinglePendingResult(CompletedOutputIterator completedOutputs) => GetSinglePendingResult(completedOutputs, out _); - internal static (Status status, TOutput output) GetSinglePendingResult(CompletedOutputIterator completedOutputs, out RecordMetadata recordMetadata) + internal static (Status status, TOutput output) GetSinglePendingResult(CompletedOutputIterator completedOutputs, out RecordMetadata recordMetadata) { ClassicAssert.IsTrue(completedOutputs.Next()); var result = (completedOutputs.Current.Status, completedOutputs.Current.Output); @@ -261,61 +268,115 @@ internal static async ValueTask DoTwoThreadRandomKeyTest(int count, bool doRando } } - internal static unsafe bool FindHashBucketEntryForKey(this TsavoriteKV store, ref TKey key, out HashBucketEntry entry) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static unsafe bool FindHashBucketEntryForKey(this TsavoriteKV store, ReadOnlySpan key, out HashBucketEntry entry) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(ref key)); + HashEntryInfo hei = new(store.storeFunctions.GetKeyHashCode64(key)); var success = store.FindTag(ref hei); entry = hei.entry; return success; } + + } + + /// Deterministic equality comparer for ints + public sealed class IntKeyComparer : IKeyComparer + { + /// + /// The default instance. + /// + /// Used to avoid allocating new comparers. + public static readonly IntKeyComparer Instance = new(); + + /// + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef() == k2.AsRef(); + + /// + public long GetHashCode64(ReadOnlySpan k) => Utility.GetHashCode(k.AsRef()); + } + + /// Deterministic equality comparer for longs + public sealed class LongKeyComparer : IKeyComparer + { + /// + /// The default instance. + /// + /// Used to avoid allocating new comparers. + public static readonly LongKeyComparer Instance = new(); + + /// + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef() == k2.AsRef(); + + /// + public long GetHashCode64(ReadOnlySpan k) => Utility.GetHashCode(k.AsRef()); } - internal class LongComparerModulo : IKeyComparer + /// Deterministic equality comparer for longs with hash modulo + internal class LongKeyComparerModulo : IKeyComparer { - readonly long mod; + internal long mod; - internal LongComparerModulo(long mod) => this.mod = mod; + internal LongKeyComparerModulo(long mod) => this.mod = mod; - public bool Equals(ref long k1, ref long k2) => k1 == k2; + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef() == k2.AsRef(); - public long GetHashCode64(ref long k) => mod == 0 ? k : k % mod; + public long GetHashCode64(ReadOnlySpan k) => mod == 0 ? k.AsRef() : k.AsRef() % mod; } - internal struct SpanByteComparerModulo : IKeyComparer + /// Deterministic equality comparer for SpanBytes with hash modulo + internal struct SpanByteKeyComparerModulo : IKeyComparer { readonly HashModulo modRange; - internal SpanByteComparerModulo(HashModulo mod) => modRange = mod; + internal SpanByteKeyComparerModulo(HashModulo mod) => modRange = mod; - public readonly bool Equals(ref SpanByte k1, ref SpanByte k2) => SpanByteComparer.StaticEquals(ref k1, ref k2); + public readonly bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => SpanByteComparer.StaticEquals(k1, k2); // Force collisions to create a chain - public readonly long GetHashCode64(ref SpanByte k) + public readonly long GetHashCode64(ReadOnlySpan k) { - var value = SpanByteComparer.StaticGetHashCode64(ref k); + var value = SpanByteComparer.StaticGetHashCode64(k); return modRange != HashModulo.NoMod ? value % (long)modRange : value; } } static class StaticTestUtils { - internal static (Status status, TOutput output) GetSinglePendingResult( - this ITsavoriteContext sessionContext) - where Functions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static (Status status, TOutput output) GetSinglePendingResult( + this ITsavoriteContext sessionContext) + where Functions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator => sessionContext.GetSinglePendingResult(out _); - internal static (Status status, TOutput output) GetSinglePendingResult( - this ITsavoriteContext sessionContext, out RecordMetadata recordMetadata) - where Functions : ISessionFunctions - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal static (Status status, TOutput output) GetSinglePendingResult( + this ITsavoriteContext sessionContext, out RecordMetadata recordMetadata) + where Functions : ISessionFunctions + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { - sessionContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); + _ = sessionContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); return TestUtils.GetSinglePendingResult(completedOutputs, out recordMetadata); } + + public static unsafe ref T AsRef(this Span spanByte) where T : unmanaged + { + Debug.Assert(spanByte.Length >= Unsafe.SizeOf(), $"Span length expected to be >= {Unsafe.SizeOf()} but was {spanByte.Length}"); + return ref Unsafe.As(ref spanByte[0]); + } + + public static ref readonly T AsRef(this ReadOnlySpan spanByte) where T : unmanaged + { + Debug.Assert(spanByte.Length >= Unsafe.SizeOf(), $"ReadOnlySpan length expected to be >= {Unsafe.SizeOf()} but was {spanByte.Length}"); + return ref MemoryMarshal.Cast(spanByte)[0]; + } + + internal static unsafe Span Set(this Span spanByte, T value) where T : unmanaged + { + ClassicAssert.IsTrue(spanByte.Length == sizeof(T)); + spanByte.AsRef() = value; + return spanByte; + } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs b/libs/storage/Tsavorite/cs/test/TransactionalUnsafeContextTests.cs similarity index 66% rename from libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs rename to libs/storage/Tsavorite/cs/test/TransactionalUnsafeContextTests.cs index 822a4a515ba..60204f53643 100644 --- a/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs +++ b/libs/storage/Tsavorite/cs/test/TransactionalUnsafeContextTests.cs @@ -15,42 +15,42 @@ using Tsavorite.test.ReadCacheTests; using static Tsavorite.test.TestUtils; -namespace Tsavorite.test.LockableUnsafeContext +namespace Tsavorite.test.TransactionalUnsafeContext { // Must be in a separate block so the "using StructStoreFunctions" is the first line in its namespace declaration. - internal class LockableUnsafeComparer : IKeyComparer + internal class TransactionalUnsafeComparer : IKeyComparer { internal int maxSleepMs; readonly Random rng = new(101); - public bool Equals(ref long k1, ref long k2) => k1 == k2; + public bool Equals(ReadOnlySpan k1, ReadOnlySpan k2) => k1.AsRef() == k2.AsRef(); - public long GetHashCode64(ref long k) + public long GetHashCode64(ReadOnlySpan k) { if (maxSleepMs > 0) Thread.Sleep(rng.Next(maxSleepMs)); - return Utility.GetHashCode(k); + return Utility.GetHashCode(k.AsRef()); } } } -namespace Tsavorite.test.LockableUnsafeContext +namespace Tsavorite.test.TransactionalUnsafeContext { - using LongAllocator = BlittableAllocator>>; - using LongStoreFunctions = StoreFunctions>; + using LongAllocator = SpanByteAllocator>; + using LongStoreFunctions = StoreFunctions; // Functions for the "Simple lock transaction" case, e.g.: // - Lock key1, key2, key3, keyResult // - Do some operation on value1, value2, value3 and write the result to valueResult - internal class LockableUnsafeFunctions : SimpleSimpleFunctions + internal class TransactionalUnsafeFunctions : SimpleLongSimpleFunctions { internal long recordAddress; - public override void PostSingleDeleter(ref long key, ref DeleteInfo deleteInfo) + public override void PostInitialDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { recordAddress = deleteInfo.Address; } - public override bool ConcurrentDeleter(ref long key, ref long value, ref DeleteInfo deleteInfo, ref RecordInfo recordInfo) + public override bool InPlaceDeleter(ref LogRecord logRecord, ref DeleteInfo deleteInfo) { recordAddress = deleteInfo.Address; return true; @@ -68,16 +68,16 @@ public BucketLockTracker() buckets = []; } - internal readonly void Increment(FixedLengthLockableKeyStruct key) => Increment(ref key); // easier with 'foreach' because iteration vars can't be passed by 'ref' - internal readonly void Increment(ref FixedLengthLockableKeyStruct key) + internal readonly void Increment(FixedLengthTransactionalKeyStruct key) => Increment(ref key); // easier with 'foreach' because iteration vars can't be passed by 'ref' + internal readonly void Increment(ref FixedLengthTransactionalKeyStruct key) { if (key.LockType == LockType.Exclusive) IncrementX(ref key); else IncrementS(ref key); } - internal readonly void Decrement(FixedLengthLockableKeyStruct key) => Decrement(ref key); - internal readonly void Decrement(ref FixedLengthLockableKeyStruct key) + internal readonly void Decrement(FixedLengthTransactionalKeyStruct key) => Decrement(ref key); + internal readonly void Decrement(ref FixedLengthTransactionalKeyStruct key) { if (key.LockType == LockType.Exclusive) DecrementX(ref key); @@ -85,12 +85,12 @@ internal readonly void Decrement(ref FixedLengthLockableKeyStruct key) DecrementS(ref key); } - internal readonly void IncrementX(ref FixedLengthLockableKeyStruct key) => AddX(ref key, 1); - internal readonly void DecrementX(ref FixedLengthLockableKeyStruct key) => AddX(ref key, -1); - internal readonly void IncrementS(ref FixedLengthLockableKeyStruct key) => AddS(ref key, 1); - internal readonly void DecrementS(ref FixedLengthLockableKeyStruct key) => AddS(ref key, -1); + internal readonly void IncrementX(ref FixedLengthTransactionalKeyStruct key) => AddX(ref key, 1); + internal readonly void DecrementX(ref FixedLengthTransactionalKeyStruct key) => AddX(ref key, -1); + internal readonly void IncrementS(ref FixedLengthTransactionalKeyStruct key) => AddS(ref key, 1); + internal readonly void DecrementS(ref FixedLengthTransactionalKeyStruct key) => AddS(ref key, -1); - private readonly void AddX(ref FixedLengthLockableKeyStruct key, int addend) + private readonly void AddX(ref FixedLengthTransactionalKeyStruct key, int addend) { if (!buckets.TryGetValue(key.KeyHash, out var counts)) counts = default; @@ -99,7 +99,7 @@ private readonly void AddX(ref FixedLengthLockableKeyStruct key, int adden buckets[key.KeyHash] = counts; } - private readonly void AddS(ref FixedLengthLockableKeyStruct key, int addend) + private readonly void AddS(ref FixedLengthTransactionalKeyStruct key, int addend) { if (!buckets.TryGetValue(key.KeyHash, out var counts)) counts = default; @@ -108,7 +108,7 @@ private readonly void AddS(ref FixedLengthLockableKeyStruct key, int adden buckets[key.KeyHash] = counts; } - internal readonly bool GetLockCounts(ref FixedLengthLockableKeyStruct key, out (int x, int s) counts) + internal readonly bool GetLockCounts(ref FixedLengthTransactionalKeyStruct key, out (int x, int s) counts) { if (!buckets.TryGetValue(key.KeyHash, out counts)) { @@ -141,7 +141,7 @@ internal readonly void AssertNoLocks() } [TestFixture] - class LockableUnsafeContextTests + class TransactionalUnsafeContextTests { const int NumRecords = 1000; const int UseNewKey = 1010; @@ -149,12 +149,12 @@ class LockableUnsafeContextTests const int ValueMult = 1_000_000; - LockableUnsafeFunctions functions; - LockableUnsafeComparer comparer; + TransactionalUnsafeFunctions functions; + TransactionalUnsafeComparer comparer; - private TsavoriteKV store; - private ClientSession session; - private BasicContext bContext; + private TsavoriteKV store; + private ClientSession session; + private BasicContext bContext; private IDevice log; [SetUp] @@ -168,7 +168,7 @@ public void Setup(bool forRecovery) } log = Devices.CreateLogDevice(Path.Combine(MethodTestDir, "test.log"), deleteOnClose: false, recoverDevice: forRecovery); - var kvSettings = new KVSettings() + var kvSettings = new KVSettings() { IndexSize = 1L << 26, LogDevice = log, @@ -195,15 +195,15 @@ public void Setup(bool forRecovery) } } - comparer = new LockableUnsafeComparer(); - functions = new LockableUnsafeFunctions(); + comparer = new TransactionalUnsafeComparer(); + functions = new TransactionalUnsafeFunctions(); store = new(kvSettings - , StoreFunctions.Create(comparer) + , StoreFunctions.Create(comparer, SpanByteRecordDisposer.Instance) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - session = store.NewSession(functions); + session = store.NewSession(functions); bContext = session.BasicContext; } @@ -227,18 +227,21 @@ public void TearDown(bool forRecovery) void Populate() { - for (int key = 0; key < NumRecords; key++) - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + for (long key = 0; key < NumRecords; key++) + { + var value = key * ValueMult; + ClassicAssert.IsFalse(bContext.Upsert(SpanByte.FromPinnedVariable(ref key), SpanByte.FromPinnedVariable(ref value)).IsPending); + } } - void AssertIsLocked(FixedLengthLockableKeyStruct key, bool xlock, bool slock) + void AssertIsLocked(FixedLengthTransactionalKeyStruct key, bool xlock, bool slock) => OverflowBucketLockTableTests.AssertLockCounts(store, ref key, xlock, slock); - void AssertIsLocked(ref FixedLengthLockableKeyStruct key, bool xlock, bool slock) + void AssertIsLocked(ref FixedLengthTransactionalKeyStruct key, bool xlock, bool slock) => OverflowBucketLockTableTests.AssertLockCounts(store, ref key, xlock, slock); void PrepareRecordLocation(FlushMode recordLocation) => PrepareRecordLocation(store, recordLocation); - static void PrepareRecordLocation(TsavoriteKV store, FlushMode recordLocation) + static void PrepareRecordLocation(TsavoriteKV store, FlushMode recordLocation) { if (recordLocation == FlushMode.ReadOnly) store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, wait: true); @@ -246,15 +249,15 @@ static void PrepareRecordLocation(TsavoriteKV luContext) + static void ClearCountsOnError(ClientSession luContext) { // If we already have an exception, clear these counts so "Run" will not report them spuriously. luContext.sharedLockCount = 0; luContext.exclusiveLockCount = 0; } - static void ClearCountsOnError(ClientSession luContext) - where TFunctions : ISessionFunctions + static void ClearCountsOnError(ClientSession luContext) + where TFunctions : ISessionFunctions { // If we already have an exception, clear these counts so "Run" will not report them spuriously. luContext.sharedLockCount = 0; @@ -284,14 +287,14 @@ void AssertNoLocks(ref BucketLockTracker blt) AssertTotalLockCounts(0, 0); } - internal void AssertBucketLockCount(ref FixedLengthLockableKeyStruct key, long expectedX, long expectedS) => OverflowBucketLockTableTests.AssertBucketLockCount(store, ref key, expectedX, expectedS); + internal void AssertBucketLockCount(ref FixedLengthTransactionalKeyStruct key, long expectedX, long expectedS) => OverflowBucketLockTableTests.AssertBucketLockCount(store, ref key, expectedX, expectedS); internal enum LockOperationType { Lock, Unlock } - internal static IEnumerable EnumActionKeyIndices(FixedLengthLockableKeyStruct[] keys, LockOperationType lockOpType) + internal static IEnumerable EnumActionKeyIndices(FixedLengthTransactionalKeyStruct[] keys, LockOperationType lockOpType) { // "Action" means the keys that will actually be locked or unlocked. - // See comments in LockableContext.DoInternalLockOp. Apps shouldn't need to do this; key sorting and enumeration + // See comments in TransactionalContext.DoInternalLockOp. Apps shouldn't need to do this; key sorting and enumeration // should be a black-box to them, so this code is just for test. if (lockOpType == LockOperationType.Lock) { @@ -319,14 +322,15 @@ public void ManualLockCollidingHashCodes([Values] UseSingleBucketComparer /* jus uint bucketIndex = 42; long genHashCode(uint uniquifier) => ((long)uniquifier << 30) | bucketIndex; - var lContext = session.LockableContext; - lContext.BeginLockable(); + var lContext = session.TransactionalContext; + lContext.BeginTransaction(); + long key1 = 101L, key2 = 102L, key3 = 103L; var keys = new[] { - new FixedLengthLockableKeyStruct(101L, genHashCode(1), LockType.Exclusive, lContext), - new FixedLengthLockableKeyStruct(102L, genHashCode(2), LockType.Exclusive, lContext), - new FixedLengthLockableKeyStruct(103L, genHashCode(3), LockType.Exclusive, lContext), + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref key1), genHashCode(1), LockType.Exclusive, lContext), + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref key2), genHashCode(2), LockType.Exclusive, lContext), + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref key3), genHashCode(3), LockType.Exclusive, lContext), }; for (var ii = 0; ii < keys.Length; ++ii) @@ -335,7 +339,7 @@ public void ManualLockCollidingHashCodes([Values] UseSingleBucketComparer /* jus lContext.Lock(keys); lContext.Unlock(keys); - lContext.EndLockable(); + lContext.EndTransaction(); } [Test] @@ -343,49 +347,51 @@ public void ManualLockCollidingHashCodes([Values] UseSingleBucketComparer /* jus [Category("Smoke")] public async Task TestShiftHeadAddressLUC([Values] CompletionSyncMode syncMode) { - long input = default; + long input = 0; const int RandSeed = 10; const int RandRange = NumRecords; const int NumRecs = 200; - Random r = new(RandSeed); + Random rng = new(RandSeed); var sw = Stopwatch.StartNew(); // Copied from UnsafeContextTests. - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new FixedLengthLockableKeyStruct[1]; + var keyVec = new FixedLengthTransactionalKeyStruct[1]; try { for (int c = 0; c < NumRecs; c++) { - keyVec[0] = new(r.Next(RandRange), LockType.Exclusive, luContext); + long rand = rng.Next(RandRange); + keyVec[0] = new(SpanByte.FromPinnedVariable(ref rand), LockType.Exclusive, luContext); luContext.Lock(keyVec); AssertBucketLockCount(ref keyVec[0], 1, 0); - var value = keyVec[0].Key + NumRecords; - _ = luContext.Upsert(ref keyVec[0].Key, ref value, Empty.Default); + var value = keyVec[0].Key.ReadOnlySpan.AsRef() + NumRecords; + _ = luContext.Upsert(keyVec[0].Key.ReadOnlySpan, SpanByte.FromPinnedVariable(ref value), Empty.Default); luContext.Unlock(keyVec); AssertBucketLockCount(ref keyVec[0], 0, 0); } AssertTotalLockCounts(0, 0); - r = new Random(RandSeed); + rng = new Random(RandSeed); sw.Restart(); for (int c = 0; c < NumRecs; c++) { - keyVec[0] = new(r.Next(RandRange), LockType.Shared, luContext); - var value = keyVec[0].Key + NumRecords; + long rand = rng.Next(RandRange); + keyVec[0] = new(SpanByte.FromPinnedVariable(ref rand), LockType.Shared, luContext); + var value = keyVec[0].Key.ReadOnlySpan.AsRef() + NumRecords; long output = 0; luContext.Lock(keyVec); AssertBucketLockCount(ref keyVec[0], 0, 1); - Status status = luContext.Read(ref keyVec[0].Key, ref input, ref output, Empty.Default); + Status status = luContext.Read(keyVec[0].Key.ReadOnlySpan, ref input, ref output, Empty.Default); luContext.Unlock(keyVec); AssertBucketLockCount(ref keyVec[0], 0, 0); ClassicAssert.IsFalse(status.IsPending); @@ -407,13 +413,21 @@ public async Task TestShiftHeadAddressLUC([Values] CompletionSyncMode syncMode) // Shift head and retry - should not find in main memory now store.Log.FlushAndEvict(true); - r = new Random(RandSeed); + rng = new Random(RandSeed); sw.Restart(); // Since we do random selection with replacement, we may not lock all keys--so need to track which we do // Similarly, we need to track bucket counts. BucketLockTracker blt = new(); - var lockKeys = Enumerable.Range(0, NumRecs).Select(ii => new FixedLengthLockableKeyStruct(r.Next(RandRange), LockType.Shared, luContext)).ToArray(); + + // Must have a pinned array to keep the key values present. + var lockLongs = GC.AllocateArray(NumRecs, pinned: true); + var lockKeys = Enumerable.Range(0, NumRecs) + .Select(ii => + { + lockLongs[ii] = rng.Next(RandRange); + return new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref lockLongs[ii]), LockType.Shared, luContext); + }).ToArray(); luContext.SortKeyHashes(lockKeys); luContext.Lock(lockKeys); @@ -423,14 +437,14 @@ public async Task TestShiftHeadAddressLUC([Values] CompletionSyncMode syncMode) ++expectedS; long output = 0; blt.IncrementS(ref lockKeys[idx]); - Status foundStatus = luContext.Read(ref lockKeys[idx].Key, ref input, ref output, Empty.Default); + Status foundStatus = luContext.Read(lockKeys[idx].Key.ReadOnlySpan, ref input, ref output, Empty.Default); ClassicAssert.IsTrue(foundStatus.IsPending); } // We did not lock all keys, only the "Action" ones - one lock per bucket, all shared in this test AssertTotalLockCounts(0, expectedS); - CompletedOutputIterator outputs; + CompletedOutputIterator outputs; if (syncMode == CompletionSyncMode.Sync) { _ = luContext.CompletePendingWithOutputs(out outputs, wait: true); @@ -455,20 +469,20 @@ public async Task TestShiftHeadAddressLUC([Values] CompletionSyncMode syncMode) while (outputs.Next()) { count++; - ClassicAssert.AreEqual(outputs.Current.Key + NumRecords, outputs.Current.Output); + ClassicAssert.AreEqual(outputs.Current.Key.AsRef() + NumRecords, outputs.Current.Output); } outputs.Dispose(); ClassicAssert.AreEqual(expectedS, count); } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget, [Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase, @@ -479,22 +493,21 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget // SetUp also reads this to determine whether to supply ReadCache settings. If ReadCache is specified it wins over CopyToTail. var useRMW = updateOp == UpdateOp.RMW; - const int readKey24 = 24, readKey51 = 51; + long readKey24 = 24, readKey51 = 51, resultValue = -1; long resultKey = resultLockTarget == ResultLockTarget.LockTable ? NumRecords + 1 : readKey24 + readKey51; - long resultValue; long expectedResult = (readKey24 + readKey51) * ValueMult; Status status; BucketLockTracker blt = new(); - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); var keys = new[] { - new FixedLengthLockableKeyStruct(readKey24, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(readKey51, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(resultKey, LockType.Exclusive, luContext), // Destination, exclusive + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref readKey24), LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref readKey51), LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref resultKey), LockType.Exclusive, luContext), // Destination, exclusive }; luContext.SortKeyHashes(keys); @@ -506,7 +519,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget // shared or exclusive), each key mapping to that bucket will report 'locked'. foreach (var key in keys) { - if (key.Key == resultKey) + if (key.Key.ReadOnlySpan.AsRef() == resultKey) AssertIsLocked(key, xlock: true, slock: false); else AssertIsLocked(key, xlock: false, slock: true); @@ -519,15 +532,16 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget // Re-get source values, to verify (e.g. they may be in readcache now). // We just locked this above, but for FlushMode.OnDisk it will be in the LockTable and will still be PENDING. - status = luContext.Read(readKey24, out var readValue24); + long output = -1; + status = luContext.Read(SpanByte.FromPinnedVariable(ref readKey24), ref output); if (flushMode == FlushMode.OnDisk) { if (status.IsPending) { _ = luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); ClassicAssert.True(completedOutputs.Next()); - readValue24 = completedOutputs.Current.Output; - ClassicAssert.AreEqual(24 * ValueMult, readValue24); + output = completedOutputs.Current.Output; + ClassicAssert.AreEqual(24 * ValueMult, output); ClassicAssert.False(completedOutputs.Next()); completedOutputs.Dispose(); } @@ -537,15 +551,15 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget ClassicAssert.IsFalse(status.IsPending, status.ToString()); } - status = luContext.Read(readKey51, out var readValue51); + status = luContext.Read(SpanByte.FromPinnedVariable(ref readKey51), ref output); if (flushMode == FlushMode.OnDisk) { if (status.IsPending) { _ = luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); ClassicAssert.True(completedOutputs.Next()); - readValue51 = completedOutputs.Current.Output; - ClassicAssert.AreEqual(51 * ValueMult, readValue51); + output = completedOutputs.Current.Output; + ClassicAssert.AreEqual(51 * ValueMult, output); ClassicAssert.False(completedOutputs.Next()); completedOutputs.Dispose(); } @@ -559,8 +573,8 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); long dummyInOut = 0; status = useRMW - ? luContext.RMW(ref resultKey, ref expectedResult, ref dummyInOut, out RecordMetadata recordMetadata) - : luContext.Upsert(ref resultKey, ref dummyInOut, ref expectedResult, ref dummyInOut, out recordMetadata); + ? luContext.RMW(SpanByte.FromPinnedVariable(ref resultKey), ref expectedResult, ref dummyInOut) + : luContext.Upsert(SpanByte.FromPinnedVariable(ref resultKey), ref dummyInOut, SpanByte.FromPinnedVariable(ref expectedResult), ref dummyInOut); if (flushMode == FlushMode.OnDisk) { if (status.IsPending) @@ -579,7 +593,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget } // Reread the destination to verify - status = luContext.Read(resultKey, out resultValue); + status = luContext.Read(SpanByte.FromPinnedVariable(ref resultKey), ref resultValue); ClassicAssert.IsFalse(status.IsPending, status.ToString()); ClassicAssert.AreEqual(expectedResult, resultValue); @@ -596,19 +610,19 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } // Verify reading the destination from the BasicContext. - status = bContext.Read(resultKey, out resultValue); + status = bContext.Read(SpanByte.FromPinnedVariable(ref resultKey), ref resultValue); ClassicAssert.IsFalse(status.IsPending, status.ToString()); ClassicAssert.AreEqual(expectedResult, resultValue); AssertTotalLockCounts(0, 0); } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase, [Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp updateOp) @@ -617,23 +631,22 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va PrepareRecordLocation(flushMode); bool initialDestWillBeLockTable = resultLockTarget == ResultLockTarget.LockTable || flushMode == FlushMode.OnDisk; - const int readKey24 = 24, readKey51 = 51, valueMult2 = 10; + long readKey24 = 24, readKey51 = 51, valueMult2 = 10, resultValue = -1; long resultKey = initialDestWillBeLockTable ? NumRecords + 1 : readKey24 + readKey51; - long resultValue; - int expectedResult = (readKey24 + readKey51) * ValueMult * valueMult2; + long expectedResult = (readKey24 + readKey51) * ValueMult * valueMult2; var useRMW = updateOp == UpdateOp.RMW; Status status; BucketLockTracker blt = new(); - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); var keys = new[] { - new FixedLengthLockableKeyStruct(readKey24, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(readKey51, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(resultKey, LockType.Exclusive, luContext), // Destination, exclusive + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref readKey24), LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref readKey51), LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref resultKey), LockType.Exclusive, luContext), // Destination, exclusive }; luContext.SortKeyHashes(keys); @@ -648,7 +661,7 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va // shared or exclusive), each key mapping to that bucket will report 'locked'. foreach (var key in keys) { - if (key.Key == resultKey) + if (key.Key.ReadOnlySpan.AsRef() == resultKey) AssertIsLocked(key, xlock: true, slock: false); else AssertIsLocked(key, xlock: false, slock: true); @@ -659,36 +672,38 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va blt.Increment(ref keys[idx]); AssertTotalLockCounts(ref blt); - status = luContext.Read(readKey24, out var readValue24); + long read24Output = 0, read51Output = 0; + status = luContext.Read(SpanByte.FromPinnedVariable(ref readKey24), ref read24Output); if (flushMode == FlushMode.OnDisk) { ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); - (status, readValue24) = GetSinglePendingResult(completedOutputs, out var recordMetadata); + (status, read24Output) = GetSinglePendingResult(completedOutputs, out var recordMetadata); ClassicAssert.IsTrue(status.Found, status.ToString()); } else ClassicAssert.IsFalse(status.IsPending, status.ToString()); - ClassicAssert.AreEqual(readKey24 * ValueMult, readValue24); + ClassicAssert.AreEqual(readKey24 * ValueMult, read24Output); // We just locked this above, but for FlushMode.OnDisk it will still be PENDING. - status = luContext.Read(readKey51, out var readValue51); + status = luContext.Read(SpanByte.FromPinnedVariable(ref readKey51), ref read51Output); if (flushMode == FlushMode.OnDisk) { ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); ClassicAssert.True(completedOutputs.Next()); - readValue51 = completedOutputs.Current.Output; + read51Output = completedOutputs.Current.Output; ClassicAssert.False(completedOutputs.Next()); completedOutputs.Dispose(); } else ClassicAssert.IsFalse(status.IsPending, status.ToString()); - ClassicAssert.AreEqual(readKey51 * ValueMult, readValue51); + ClassicAssert.AreEqual(readKey51 * ValueMult, read51Output); if (!initialDestWillBeLockTable) { - status = luContext.Read(resultKey, out var initialResultValue); + long initialResultValue = 0; + status = luContext.Read(SpanByte.FromPinnedVariable(ref resultKey), ref initialResultValue); if (flushMode == FlushMode.OnDisk) { ClassicAssert.IsTrue(status.IsPending, status.ToString()); @@ -703,12 +718,13 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va // Set the phase to Phase.INTERMEDIATE to test the non-Phase.REST blocks session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); + resultValue = (read24Output + read51Output) * valueMult2; status = useRMW - ? luContext.RMW(resultKey, (readValue24 + readValue51) * valueMult2) - : luContext.Upsert(resultKey, (readValue24 + readValue51) * valueMult2); + ? luContext.RMW(SpanByte.FromPinnedVariable(ref resultKey), ref resultValue) // value is 'input' for RMW + : luContext.Upsert(SpanByte.FromPinnedVariable(ref resultKey), SpanByte.FromPinnedVariable(ref resultValue)); ClassicAssert.IsFalse(status.IsPending, status.ToString()); - status = luContext.Read(resultKey, out resultValue); + status = luContext.Read(SpanByte.FromPinnedVariable(ref resultKey), ref resultValue); ClassicAssert.IsFalse(status.IsPending, status.ToString()); ClassicAssert.AreEqual(expectedResult, resultValue); @@ -725,19 +741,20 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } // Verify from the full Basic Context - status = bContext.Read(resultKey, out resultValue); + var value = 0L; + status = bContext.Read(SpanByte.FromPinnedVariable(ref resultKey), ref value); ClassicAssert.IsFalse(status.IsPending, status.ToString()); ClassicAssert.AreEqual(expectedResult, resultValue); AssertTotalLockCounts(0, 0); } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] #pragma warning disable IDE0060 // Remove unused parameter: readCopyDestination is used by Setup public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Values] ReadCopyDestination readCopyDestination, @@ -751,14 +768,15 @@ public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Valu BucketLockTracker blt = new(); // SetUp also reads this to determine whether to supply ReadCache settings. If ReadCache is specified it wins over CopyToTail. - long resultKey = resultLockTarget == ResultLockTarget.LockTable ? NumRecords + 1 : 75; + long resultKeyVal = resultLockTarget == ResultLockTarget.LockTable ? NumRecords + 1 : 75, output = 0, resultValue = -1; + Span resultKey = SpanByte.FromPinnedVariable(ref resultKeyVal); Status status; - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new[] { new FixedLengthLockableKeyStruct(resultKey, LockType.Exclusive, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(resultKey, LockType.Exclusive, luContext) }; try { @@ -771,11 +789,11 @@ public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Valu // Set the phase to Phase.INTERMEDIATE to test the non-Phase.REST blocks session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); - status = luContext.Delete(ref resultKey); + status = luContext.Delete(resultKey); ClassicAssert.IsFalse(status.IsPending, status.ToString()); // Reread the destination to verify - status = luContext.Read(resultKey, out var _); + status = luContext.Read(resultKey, ref output); ClassicAssert.IsFalse(status.Found, status.ToString()); luContext.Unlock(keyVec); @@ -790,18 +808,18 @@ public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Valu } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } // Verify reading the destination from the full Basic Context - status = bContext.Read(resultKey, out var _); + status = bContext.Read(resultKey, ref resultValue); ClassicAssert.IsFalse(status.Found, status.ToString()); AssertTotalLockCounts(0, 0); } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void StressManualLocks([Values(1, 8)] int numLockThreads, [Values(0, 1, 8)] int numOpThreads) { @@ -813,7 +831,7 @@ public void StressManualLocks([Values(1, 8)] int numLockThreads, [Values(0, 1, 8 const int numIncrement = 5; const int numIterations = 1000; - IEnumerable enumKeys(Random rng) + IEnumerable enumKeys(Random rng) { for (var key = baseKey + rng.Next(numIncrement); key < baseKey + numKeys; key += rng.Next(1, numIncrement)) yield return key; @@ -825,29 +843,30 @@ void runManualLockThread(int tid) Random rng = new(tid + 101); - using var localSession = store.NewSession(new LockableUnsafeFunctions()); - var luContext = localSession.LockableUnsafeContext; + using var localSession = store.NewSession(new TransactionalUnsafeFunctions()); + var luContext = localSession.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - IEnumerable> enumKeysToLock() + IEnumerable<(long key, LockType lockType)> enumKeysToLock() { foreach (var key in enumKeys(rng)) - { - var lockType = rng.Next(100) < 60 ? LockType.Shared : LockType.Exclusive; - yield return new(key, lockType, luContext); - } + yield return (key, rng.Next(100) < 60 ? LockType.Shared : LockType.Exclusive); } for (var iteration = 0; iteration < numIterations; ++iteration) { - var keys = enumKeysToLock().ToArray(); - FixedLengthLockableKeyStruct.Sort(keys, luContext); + var keysToLock = enumKeysToLock().ToArray(); + var keys = GC.AllocateArray(keysToLock.Length, pinned: true); + for (var ii = 0; ii < keysToLock.Length; ++ii) + keys[ii] = new(SpanByte.FromPinnedVariable(ref keysToLock[ii].key), keysToLock[ii].lockType, luContext); + + FixedLengthTransactionalKeyStruct.Sort(keys, luContext); luContext.Lock(keys); luContext.Unlock(keys); } - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } @@ -855,18 +874,21 @@ void runLTransientLockOpThread(int tid) { Random rng = new(tid + 101); - using var localSession = store.NewSession(new LockableUnsafeFunctions()); + using var localSession = store.NewSession(new TransactionalUnsafeFunctions()); var basicContext = localSession.BasicContext; for (var iteration = 0; iteration < numIterations; ++iteration) { - foreach (var key in enumKeys(rng)) + foreach (var key0 in enumKeys(rng)) + { + long key = key0, value = key * ValueMult; _ = rng.Next(100) switch { - int rand when rand < 33 => basicContext.Read(key).status, - int rand when rand < 66 => basicContext.Upsert(key, key * ValueMult), - _ => basicContext.RMW(key, key * ValueMult) + int rand when rand < 33 => basicContext.Read(SpanByte.FromPinnedVariable(ref key)).status, + int rand when rand < 66 => basicContext.Upsert(SpanByte.FromPinnedVariable(ref key), SpanByte.FromPinnedVariable(ref value)), + _ => basicContext.RMW(SpanByte.FromPinnedVariable(ref key), ref value) }; + } } } @@ -886,13 +908,13 @@ void runLTransientLockOpThread(int tid) AssertTotalLockCounts(0, 0); } - FixedLengthLockableKeyStruct AddLockTableEntry(LockableUnsafeContext luContext, long key) - where TFunctions : ISessionFunctions + FixedLengthTransactionalKeyStruct AddLockTableEntry(TransactionalUnsafeContext luContext, ReadOnlySpan key) + where TFunctions : ISessionFunctions { - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Exclusive, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key, LockType.Exclusive, luContext) }; luContext.Lock(keyVec); - HashEntryInfo hei = new(comparer.GetHashCode64(ref key)); + HashEntryInfo hei = new(comparer.GetHashCode64(key)); PopulateHei(ref hei); var lockState = store.LockTable.GetLockState(ref hei); @@ -902,41 +924,42 @@ FixedLengthLockableKeyStruct AddLockTableEntry(LockableUnsafeC return keyVec[0]; } - void VerifyAndUnlockSplicedInKey(LockableUnsafeContext luContext, long expectedKey) - where TFunctions : ISessionFunctions + void VerifyAndUnlockSplicedInKey(TransactionalUnsafeContext luContext, ReadOnlySpan expectedKey) + where TFunctions : ISessionFunctions { // Scan to the end of the readcache chain and verify we inserted the value. var (_, pa) = ChainTests.SkipReadCacheChain(store, expectedKey); - var storedKey = store.hlog.GetKey(pa); - ClassicAssert.AreEqual(expectedKey, storedKey); + var storedKey = LogRecord.GetInlineKey(pa); + ClassicAssert.AreEqual(expectedKey.AsRef(), storedKey.AsRef()); - var keyVec = new[] { new FixedLengthLockableKeyStruct(expectedKey, LockType.Exclusive, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(expectedKey, LockType.Exclusive, luContext) }; luContext.Unlock(keyVec); } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyLocksAfterReadAndCTTTest() { Populate(); store.Log.FlushAndEvict(wait: true); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; - long input = 0, output = 0, key = 24; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; + long input = 0, output = 0, keyVal = 24; + var key = SpanByte.FromPinnedVariable(ref keyVal); ReadOptions readOptions = new() { CopyOptions = new(ReadCopyFrom.AllImmutable, ReadCopyTo.MainLog) }; BucketLockTracker blt = new(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); try { var keyStruct = AddLockTableEntry(luContext, key); blt.Increment(ref keyStruct); AssertTotalLockCounts(ref blt); - var status = luContext.Read(ref key, ref input, ref output, ref readOptions, out _); + var status = luContext.Read(key, ref input, ref output, ref readOptions, out _); ClassicAssert.IsTrue(status.IsPending, status.ToString()); _ = luContext.CompletePending(wait: true); @@ -951,28 +974,28 @@ public void VerifyLocksAfterReadAndCTTTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyCountsAfterFlushAndEvict() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; BucketLockTracker blt = new(); long key = 24; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); try { - var keyVec = new[] { new FixedLengthLockableKeyStruct(key, LockType.Exclusive, luContext) }; + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(SpanByte.FromPinnedVariable(ref key), LockType.Exclusive, luContext) }; luContext.Lock(keyVec); blt.Increment(ref keyVec[0]); AssertTotalLockCounts(ref blt); @@ -993,7 +1016,7 @@ public void VerifyCountsAfterFlushAndEvict() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } @@ -1009,30 +1032,33 @@ void PopulateAndEvict(bool immutable = false) } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyCountAfterUpsertToTailTest([Values] ChainTests.RecordRegion recordRegion) { PopulateAndEvict(recordRegion == ChainTests.RecordRegion.Immutable); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; BucketLockTracker blt = new(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - FixedLengthLockableKeyStruct keyStruct = default; + FixedLengthTransactionalKeyStruct keyStruct = default; try { + long keyNum = 0; + var key = SpanByte.FromPinnedVariable(ref keyNum); if (recordRegion is ChainTests.RecordRegion.Immutable or ChainTests.RecordRegion.OnDisk) - keyStruct = AddLockTableEntry(luContext, UseExistingKey); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseExistingKey)); else - keyStruct = AddLockTableEntry(luContext, UseNewKey); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseNewKey)); + blt.Increment(ref keyStruct); - var status = luContext.Upsert(keyStruct.Key, keyStruct.Key * ValueMult); + var status = luContext.Upsert(key, key); ClassicAssert.IsTrue(status.Record.Created, status.ToString()); - VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key); + VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key.ReadOnlySpan); blt.Decrement(ref keyStruct); AssertNoLocks(ref blt); } @@ -1043,43 +1069,47 @@ public void VerifyCountAfterUpsertToTailTest([Values] ChainTests.RecordRegion re } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyCountAfterRMWToTailTest([Values] ChainTests.RecordRegion recordRegion) { PopulateAndEvict(recordRegion == ChainTests.RecordRegion.Immutable); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; BucketLockTracker blt = new(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - FixedLengthLockableKeyStruct keyStruct = default; + FixedLengthTransactionalKeyStruct keyStruct = default; try { + long keyVal = 0; + var key = SpanByte.FromPinnedVariable(ref keyVal); if (recordRegion is ChainTests.RecordRegion.Immutable or ChainTests.RecordRegion.OnDisk) { - keyStruct = AddLockTableEntry(luContext, UseExistingKey); - var status = luContext.RMW(keyStruct.Key, keyStruct.Key * ValueMult); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseExistingKey)); + var input = keyStruct.Key.ReadOnlySpan.AsRef() * ValueMult; + var status = luContext.RMW(keyStruct.Key.ReadOnlySpan, ref input); ClassicAssert.IsTrue(recordRegion == ChainTests.RecordRegion.OnDisk ? status.IsPending : status.Found); _ = luContext.CompletePending(wait: true); } else { - keyStruct = AddLockTableEntry(luContext, UseNewKey); - var status = luContext.RMW(keyStruct.Key, keyStruct.Key * ValueMult); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseNewKey)); + var input = keyStruct.Key.ReadOnlySpan.AsRef() * ValueMult; + var status = luContext.RMW(keyStruct.Key.ReadOnlySpan, ref input); ClassicAssert.IsFalse(status.Found, status.ToString()); } blt.Increment(ref keyStruct); - VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key); + VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key.ReadOnlySpan); blt.Decrement(ref keyStruct); AssertNoLocks(ref blt); } @@ -1090,45 +1120,47 @@ public void VerifyCountAfterRMWToTailTest([Values] ChainTests.RecordRegion recor } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyCountAfterDeleteToTailTest([Values] ChainTests.RecordRegion recordRegion) { PopulateAndEvict(recordRegion == ChainTests.RecordRegion.Immutable); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; BucketLockTracker blt = new(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - FixedLengthLockableKeyStruct keyStruct = default; + FixedLengthTransactionalKeyStruct keyStruct = default; try { + long keyVal = 0; + var key = SpanByte.FromPinnedVariable(ref keyVal); if (recordRegion is ChainTests.RecordRegion.Immutable or ChainTests.RecordRegion.OnDisk) { - keyStruct = AddLockTableEntry(luContext, UseExistingKey); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseExistingKey)); blt.Increment(ref keyStruct); - var status = luContext.Delete(keyStruct.Key); + var status = luContext.Delete(keyStruct.Key.ReadOnlySpan); // Delete does not search outside mutable region so the key will not be found ClassicAssert.IsTrue(!status.Found && status.Record.Created, status.ToString()); } else { - keyStruct = AddLockTableEntry(luContext, UseNewKey); + keyStruct = AddLockTableEntry(luContext, key.Set((long)UseNewKey)); blt.Increment(ref keyStruct); - var status = luContext.Delete(keyStruct.Key); + var status = luContext.Delete(keyStruct.Key.ReadOnlySpan); ClassicAssert.IsFalse(status.Found, status.ToString()); } - VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key); + VerifyAndUnlockSplicedInKey(luContext, keyStruct.Key.ReadOnlySpan); blt.Decrement(ref keyStruct); AssertNoLocks(ref blt); } @@ -1139,28 +1171,31 @@ public void VerifyCountAfterDeleteToTailTest([Values] ChainTests.RecordRegion re } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void LockAndUnlockInLockTableOnlyTest() { // For this, just don't load anything, and it will happen in lock table. - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; BucketLockTracker blt = new(); - FixedLengthLockableKeyStruct createKey(long key) => new(key, (key & 1) == 0 ? LockType.Exclusive : LockType.Shared, luContext); + FixedLengthTransactionalKeyStruct createKey(ReadOnlySpan key) => new(key, (key.AsRef() & 1) == 0 ? LockType.Exclusive : LockType.Shared, luContext); + + // Need a pinned array for SpanByteFrom + var keys = GC.AllocateArray(NumRecords, pinned: true); var rng = new Random(101); - var keyVec = Enumerable.Range(0, NumRecords).Select(ii => createKey(rng.Next(NumRecords))).ToArray(); + var keyVec = Enumerable.Range(0, NumRecords).Select(ii => { keys[ii] = rng.Next(NumRecords); return createKey(SpanByte.FromPinnedVariable(ref keys[ii]));}).ToArray(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); try { store.LockTable.SortKeyHashes(keyVec); @@ -1194,13 +1229,13 @@ public void LockAndUnlockInLockTableOnlyTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void VerifyCountAfterReadOnlyToUpdateRecordTest([Values] UpdateOp updateOp) { @@ -1209,21 +1244,26 @@ public void VerifyCountAfterReadOnlyToUpdateRecordTest([Values] UpdateOp updateO static long getValue(long key) => key + ValueMult; - var luContext = session.LockableUnsafeContext; + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new[] { new FixedLengthLockableKeyStruct(42, LockType.Exclusive, luContext) }; + long key42Val = 42L; + var key42 = SpanByte.FromPinnedVariable(ref key42Val); + var keyVec = new[] { new FixedLengthTransactionalKeyStruct(key42, LockType.Exclusive, luContext) }; try { luContext.Lock(keyVec); + long valueVal = getValue(key42Val); + var value = SpanByte.FromPinnedVariable(ref valueVal); + var status = updateOp switch { - UpdateOp.Upsert => luContext.Upsert(keyVec[0].Key, getValue(keyVec[0].Key)), - UpdateOp.RMW => luContext.RMW(keyVec[0].Key, getValue(keyVec[0].Key)), - UpdateOp.Delete => luContext.Delete(keyVec[0].Key), + UpdateOp.Upsert => luContext.Upsert(key42, value), + UpdateOp.RMW => luContext.RMW(keyVec[0].Key.ReadOnlySpan, ref valueVal), + UpdateOp.Delete => luContext.Delete(key42), _ => new(StatusCode.Error) }; ClassicAssert.IsFalse(status.IsFaulted, $"Unexpected UpdateOp {updateOp}, status {status}"); @@ -1232,10 +1272,10 @@ public void VerifyCountAfterReadOnlyToUpdateRecordTest([Values] UpdateOp updateO else ClassicAssert.IsTrue(status.Record.Created, status.ToString()); - OverflowBucketLockTableTests.AssertLockCounts(store, keyVec[0].Key, true, 0); + OverflowBucketLockTableTests.AssertLockCounts(store, keyVec[0].Key.ReadOnlySpan, true, 0); luContext.Unlock(keyVec); - OverflowBucketLockTableTests.AssertLockCounts(store, keyVec[0].Key, false, 0); + OverflowBucketLockTableTests.AssertLockCounts(store, keyVec[0].Key.ReadOnlySpan, false, 0); } catch (Exception) { @@ -1244,27 +1284,31 @@ public void VerifyCountAfterReadOnlyToUpdateRecordTest([Values] UpdateOp updateO } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] public void LockNewRecordThenUpdateAndUnlockTest([Values] UpdateOp updateOp) { const int numNewRecords = 100; - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; - int getValue(int key) => key + ValueMult; + long getValue(long kk) => kk + ValueMult; + + long keyVal = 0, valueVal = 0; + Span key = SpanByte.FromPinnedVariable(ref keyVal); + Span value = SpanByte.FromPinnedVariable(ref valueVal); // If we are testing Delete, then we need to have the records ON-DISK first; Delete is a no-op for unfound records. if (updateOp == UpdateOp.Delete) { - for (var key = NumRecords; key < NumRecords + numNewRecords; ++key) - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + for (long keyNum = NumRecords; keyNum < NumRecords + numNewRecords; ++keyNum) + ClassicAssert.IsFalse(bContext.Upsert(key.Set(keyNum), value.Set(keyVal * ValueMult)).IsPending); store.Log.FlushAndEvict(wait: true); } @@ -1273,18 +1317,18 @@ public void LockNewRecordThenUpdateAndUnlockTest([Values] UpdateOp updateOp) BucketLockTracker blt = new(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new FixedLengthLockableKeyStruct[1]; + var keyVec = new FixedLengthTransactionalKeyStruct[1]; try { // We don't sleep in this test comparer.maxSleepMs = 0; - for (var key = NumRecords; key < NumRecords + numNewRecords; ++key) + for (long keyNum = NumRecords; keyNum < NumRecords + numNewRecords; ++keyNum) { - keyVec[0] = new(key, LockType.Exclusive, luContext); + keyVec[0] = new(key.Set(keyNum), LockType.Exclusive, luContext); luContext.Lock(keyVec); for (var iter = 0; iter < 2; ++iter) { @@ -1302,26 +1346,27 @@ public void LockNewRecordThenUpdateAndUnlockTest([Values] UpdateOp updateOp) } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } - void updater(int key, int iter) + void updater(ReadOnlySpan key, int iter) { + var localValueNum = getValue(key.AsRef()); try { Status status; switch (updateOp) { case UpdateOp.Upsert: - status = luContext.Upsert(key, getValue(key)); + status = luContext.Upsert(key, SpanByte.FromPinnedVariable(ref localValueNum)); if (iter == 0) ClassicAssert.IsTrue(status.NotFound && status.Record.Created, status.ToString()); else ClassicAssert.IsTrue(status.Found && status.Record.InPlaceUpdated, status.ToString()); break; case UpdateOp.RMW: - status = luContext.RMW(key, getValue(key)); + status = luContext.RMW(key, ref localValueNum); if (iter == 0) ClassicAssert.IsTrue(status.NotFound && status.Record.Created, status.ToString()); else @@ -1337,6 +1382,7 @@ void updater(int key, int iter) Assert.Fail($"Unexpected updateOp {updateOp}"); return; } + ; ClassicAssert.IsFalse(status.IsFaulted, $"Unexpected UpdateOp {updateOp}, status {status}"); } catch (Exception) @@ -1348,7 +1394,7 @@ void updater(int key, int iter) } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] //[Repeat(100)] public void LockNewRecordThenUnlockThenUpdateTest([Values] UpdateOp updateOp) { @@ -1357,19 +1403,20 @@ public void LockNewRecordThenUnlockThenUpdateTest([Values] UpdateOp updateOp) const int numNewRecords = 50; - using var lockSession = store.NewSession>(new SimpleSimpleFunctions()); - var lockLuContext = lockSession.LockableUnsafeContext; + using var lockSession = store.NewSession(new SimpleLongSimpleFunctions()); + var lockLuContext = lockSession.TransactionalUnsafeContext; - using var updateSession = store.NewSession>(new SimpleSimpleFunctions()); + using var updateSession = store.NewSession(new SimpleLongSimpleFunctions()); var basicContext = updateSession.BasicContext; - int getValue(int key) => key + ValueMult; + long getValue(long kk) => kk + ValueMult; // If we are testing Delete, then we need to have the records ON-DISK first; Delete is a no-op for unfound records. + // The actual value here is not important as we don't test it later. if (updateOp == UpdateOp.Delete) { - for (var key = NumRecords; key < NumRecords + numNewRecords; ++key) - ClassicAssert.IsFalse(bContext.Upsert(key, key * ValueMult).IsPending); + for (long keyNum = NumRecords; keyNum < NumRecords + numNewRecords; ++keyNum) + ClassicAssert.IsFalse(bContext.Upsert(SpanByte.FromPinnedVariable(ref keyNum), SpanByte.FromPinnedVariable(ref keyNum)).IsPending); store.Log.FlushAndEvict(wait: true); } @@ -1377,26 +1424,26 @@ public void LockNewRecordThenUnlockThenUpdateTest([Values] UpdateOp updateOp) Populate(); lockLuContext.BeginUnsafe(); - lockLuContext.BeginLockable(); + lockLuContext.BeginTransaction(); // These are for debugging - int[] lastLockerKeys = new int[6], lastUpdaterKeys = new int[3]; + long[] lastLockerKeys = new long[6], lastUpdaterKeys = new long[3]; // Randomize the start and lock-hold wait times int maxSleepMs = 10; Random lockRng = new(101), updateRng = new(107); - var lockKeyVec = new FixedLengthLockableKeyStruct[1]; + var lockKeyVec = new FixedLengthTransactionalKeyStruct[1]; try { - for (var key = NumRecords; key < NumRecords + numNewRecords; ++key) + for (long keyNum = NumRecords; keyNum < NumRecords + numNewRecords; ++keyNum) { for (var iter = 0; iter < 2; ++iter) { // Use Task instead of Thread because this propagates exceptions (such as Assert.* failures) back to this thread. // BasicContext's transient lock will wait for the lock/unlock combo to complete, or the lock/unlock will wait for basicContext to finish if it wins. - Task.WaitAll(Task.Run(() => locker(key)), Task.Run(() => updater(key, iter))); + Task.WaitAll(Task.Run(() => locker(keyNum)), Task.Run(() => updater(keyNum, iter))); } AssertBucketLockCount(ref lockKeyVec[0], 0, 0); @@ -1409,29 +1456,29 @@ public void LockNewRecordThenUnlockThenUpdateTest([Values] UpdateOp updateOp) } finally { - lockLuContext.EndLockable(); + lockLuContext.EndTransaction(); lockLuContext.EndUnsafe(); } - void locker(int key) + void locker(long keyNum) { - lockKeyVec[0] = new(key, LockType.Exclusive, lockLuContext); + lockKeyVec[0] = new(SpanByte.FromPinnedVariable(ref keyNum), LockType.Exclusive, lockLuContext); try { - // Begin/EndLockable are called outside this function; we could not EndLockable in here as the lock lifetime is beyond that. - // (BeginLockable's scope is the session; BeginUnsafe's scope is the thread. The session is still "mono-threaded" here because + // Begin/EndTransaction are called outside this function; we could not EndTransaction in here as the lock lifetime is beyond that. + // (BeginTransaction's scope is the session; BeginUnsafe's scope is the thread. The session is still "mono-threaded" here because // only one thread at a time is making calls on it.) - lastLockerKeys[0] = key; + lastLockerKeys[0] = keyNum; lockLuContext.BeginUnsafe(); - lastLockerKeys[1] = key; + lastLockerKeys[1] = keyNum; Thread.Sleep(lockRng.Next(maxSleepMs)); - lastLockerKeys[2] = key; + lastLockerKeys[2] = keyNum; lockLuContext.Lock(lockKeyVec); - lastLockerKeys[3] = key; + lastLockerKeys[3] = keyNum; Thread.Sleep(lockRng.Next(maxSleepMs)); - lastLockerKeys[4] = key; + lastLockerKeys[4] = keyNum; lockLuContext.Unlock(lockKeyVec); - lastLockerKeys[5] = key; + lastLockerKeys[5] = keyNum; } catch (Exception) { @@ -1444,25 +1491,27 @@ void locker(int key) } } - void updater(int key, int iter) + void updater(long keyNum, int iter) { try { - lastUpdaterKeys[0] = key; + lastUpdaterKeys[0] = keyNum; Thread.Sleep(updateRng.Next(maxSleepMs)); - lastUpdaterKeys[1] = key; + lastUpdaterKeys[1] = keyNum; Status status; + var localValueNum = getValue(keyNum); + Span key = SpanByte.FromPinnedVariable(ref keyNum), value = SpanByte.FromPinnedVariable(ref localValueNum); switch (updateOp) { case UpdateOp.Upsert: - status = basicContext.Upsert(key, getValue(key)); + status = basicContext.Upsert(key, value); if (iter == 0) ClassicAssert.IsTrue(status.NotFound && status.Record.Created, status.ToString()); else ClassicAssert.IsTrue(status.Found && status.Record.InPlaceUpdated, status.ToString()); break; case UpdateOp.RMW: - status = basicContext.RMW(key, getValue(key)); + status = basicContext.RMW(key, ref localValueNum); if (iter == 0) ClassicAssert.IsTrue(status.NotFound && status.Record.Created, status.ToString()); else @@ -1480,7 +1529,7 @@ void updater(int key, int iter) } ; ClassicAssert.IsFalse(status.IsFaulted, $"Unexpected UpdateOp {updateOp}, status {status}"); - lastUpdaterKeys[2] = key; + lastUpdaterKeys[2] = keyNum; } catch (Exception) { @@ -1491,22 +1540,23 @@ void updater(int key, int iter) } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void MultiSharedLockTest() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; - const int key = 42; + long keyNum = 42; + var key = SpanByte.FromPinnedVariable(ref keyNum); var maxLocks = 63; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new FixedLengthLockableKeyStruct[1]; + var keyVec = new FixedLengthTransactionalKeyStruct[1]; try { @@ -1532,36 +1582,41 @@ public void MultiSharedLockTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void TryLockTimeSpanLimitTest() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var keyVec = new FixedLengthLockableKeyStruct[] + var keys = GC.AllocateArray(3, pinned: true); + keys[0] = 42; + keys[1] = 43; + keys[2] = 44; + + var keyVec = new FixedLengthTransactionalKeyStruct[] { - new(42, LockType.Exclusive, luContext), - new(43, LockType.Exclusive, luContext), - new(44, LockType.Exclusive, luContext) + new(SpanByte.FromPinnedVariable(ref keys[0]), LockType.Exclusive, luContext), + new(SpanByte.FromPinnedVariable(ref keys[1]), LockType.Exclusive, luContext), + new(SpanByte.FromPinnedVariable(ref keys[2]), LockType.Exclusive, luContext) }; // First ensure things work with no blocking locks. ClassicAssert.IsTrue(luContext.TryLock(keyVec)); luContext.Unlock(keyVec); - var blockingVec = new FixedLengthLockableKeyStruct[1]; + var blockingVec = new FixedLengthTransactionalKeyStruct[1]; try { @@ -1575,8 +1630,8 @@ public void TryLockTimeSpanLimitTest() ClassicAssert.IsFalse(luContext.TryLock(keyVec, TimeSpan.FromMilliseconds(20))); foreach (var k in keyVec) { - if (k.Key != blockingVec[0].Key) - OverflowBucketLockTableTests.AssertLockCounts(store, k.Key, false, 0); + if (k.Key.ReadOnlySpan.AsRef() != blockingVec[0].Key.ReadOnlySpan.AsRef()) + OverflowBucketLockTableTests.AssertLockCounts(store, k.Key.ReadOnlySpan, false, 0); } luContext.Unlock(blockingVec); @@ -1589,36 +1644,41 @@ public void TryLockTimeSpanLimitTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void TryLockCancellationTest() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); + + var keys = GC.AllocateArray(3, pinned: true); + keys[0] = 42; + keys[1] = 43; + keys[2] = 44; - var keyVec = new FixedLengthLockableKeyStruct[] + var keyVec = new FixedLengthTransactionalKeyStruct[] { - new(42, LockType.Exclusive, luContext), - new(43, LockType.Exclusive, luContext), - new(44, LockType.Exclusive, luContext) + new(SpanByte.FromPinnedVariable(ref keys[0]), LockType.Exclusive, luContext), + new(SpanByte.FromPinnedVariable(ref keys[1]), LockType.Exclusive, luContext), + new(SpanByte.FromPinnedVariable(ref keys[2]), LockType.Exclusive, luContext) }; // First ensure things work with no blocking locks. ClassicAssert.IsTrue(luContext.TryLock(keyVec)); luContext.Unlock(keyVec); - var blockingVec = new FixedLengthLockableKeyStruct[1]; + var blockingVec = new FixedLengthTransactionalKeyStruct[1]; try { @@ -1634,8 +1694,8 @@ public void TryLockCancellationTest() ClassicAssert.IsFalse(luContext.TryLock(keyVec, cts.Token)); foreach (var k in keyVec) { - if (k.Key != blockingVec[0].Key) - OverflowBucketLockTableTests.AssertLockCounts(store, k.Key, false, 0); + if (k.Key.ReadOnlySpan.AsRef() != blockingVec[0].Key.ReadOnlySpan.AsRef()) + OverflowBucketLockTableTests.AssertLockCounts(store, k.Key.ReadOnlySpan, false, 0); } luContext.Unlock(blockingVec); @@ -1648,28 +1708,29 @@ public void TryLockCancellationTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void TryPromoteLockTimeSpanLimitTest() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var key = 42; + long keyNum = 42; + var key = SpanByte.FromPinnedVariable(ref keyNum); - var exclusiveVec = new FixedLengthLockableKeyStruct[] { new(key, LockType.Exclusive, luContext) }; - var sharedVec = new FixedLengthLockableKeyStruct[] { new(key, LockType.Shared, luContext) }; + var exclusiveVec = new FixedLengthTransactionalKeyStruct[] { new(key, LockType.Exclusive, luContext) }; + var sharedVec = new FixedLengthTransactionalKeyStruct[] { new(key, LockType.Shared, luContext) }; try { @@ -1691,28 +1752,29 @@ public void TryPromoteLockTimeSpanLimitTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } [Test] - [Category(LockableUnsafeContextTestCategory)] + [Category(TransactionalUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void TryPromoteLockCancellationTest() { Populate(); - using var session = store.NewSession>(new SimpleSimpleFunctions()); - var luContext = session.LockableUnsafeContext; + using var session = store.NewSession(new SimpleLongSimpleFunctions()); + var luContext = session.TransactionalUnsafeContext; luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); - var key = 42; + long keyNum = 42; + var key = SpanByte.FromPinnedVariable(ref keyNum); - var exclusiveVec = new FixedLengthLockableKeyStruct[] { new(key, LockType.Exclusive, luContext) }; - var sharedVec = new FixedLengthLockableKeyStruct[] { new(key, LockType.Shared, luContext) }; + var exclusiveVec = new FixedLengthTransactionalKeyStruct[] { new(key, LockType.Exclusive, luContext) }; + var sharedVec = new FixedLengthTransactionalKeyStruct[] { new(key, LockType.Shared, luContext) }; try { @@ -1735,7 +1797,7 @@ public void TryPromoteLockCancellationTest() } finally { - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); } } diff --git a/libs/storage/Tsavorite/cs/test/UnsafeContextTests.cs b/libs/storage/Tsavorite/cs/test/UnsafeContextTests.cs index c4888222d32..3ce458a3e51 100644 --- a/libs/storage/Tsavorite/cs/test/UnsafeContextTests.cs +++ b/libs/storage/Tsavorite/cs/test/UnsafeContextTests.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Diagnostics; using System.IO; @@ -34,7 +36,7 @@ public void Setup() DeleteDirectory(MethodTestDir, wait: true); } - private void Setup(KVSettings kvSettings, DeviceType deviceType) + private void Setup(KVSettings kvSettings, DeviceType deviceType) { string filename = Path.Join(MethodTestDir, TestContext.CurrentContext.Test.Name + deviceType.ToString() + ".log"); log = CreateTestDevice(deviceType, filename); @@ -623,4 +625,6 @@ public void ReadBareMinParams([Values] DeviceType deviceType) } } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/libs/storage/Tsavorite/cs/test/VLVector.cs b/libs/storage/Tsavorite/cs/test/VLVector.cs index a6bcc134f17..266a3a9a7ae 100644 --- a/libs/storage/Tsavorite/cs/test/VLVector.cs +++ b/libs/storage/Tsavorite/cs/test/VLVector.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +#if LOGRECORD_TODO + using System; using System.Runtime.CompilerServices; using NUnit.Framework.Legacy; @@ -49,27 +51,23 @@ public override void ReadCompletionCallback(ref SpanByte key, ref SpanByte input } // Read functions - public override bool SingleReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref int[] dst, ref ReadInfo readInfo) - { - dst = value.ToArray(); - return true; - } - - public override bool ConcurrentReader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref int[] dst, ref ReadInfo readInfo, ref RecordInfo recordInfo) + public override bool Reader(ref SpanByte key, ref SpanByte input, ref SpanByte value, ref int[] dst, ref ReadInfo readInfo) { dst = value.ToArray(); return true; } // Upsert functions - public override bool SingleWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref int[] output, ref UpsertInfo upsertInfo, WriteReason reason, ref RecordInfo recordInfo) - => base.SingleWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); + public override bool InitialWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref int[] output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + => base.InitialWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, reason, ref recordInfo); - public override bool ConcurrentWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref int[] output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) + public override bool InPlaceWriter(ref SpanByte key, ref SpanByte input, ref SpanByte src, ref SpanByte dst, ref int[] output, ref UpsertInfo upsertInfo, ref RecordInfo recordInfo) { if (src.Length != dst.Length) return false; - return base.ConcurrentWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo); + return base.InPlaceWriter(ref key, ref input, ref src, ref dst, ref output, ref upsertInfo, ref recordInfo); } } -} \ No newline at end of file +} + +#endif // LOGRECORD_TODO diff --git a/main/GarnetServer/Extensions/GetTwoKeysNoTxn.cs b/main/GarnetServer/Extensions/GetTwoKeysNoTxn.cs index 3eb64518f92..f0b88f0f314 100644 --- a/main/GarnetServer/Extensions/GetTwoKeysNoTxn.cs +++ b/main/GarnetServer/Extensions/GetTwoKeysNoTxn.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet { @@ -37,8 +38,8 @@ public override void Finalize(TGarnetApi api, ref CustomProcedureInp var key1 = GetNextArg(ref procInput, ref offset); var key2 = GetNextArg(ref procInput, ref offset); - api.GET(key1, out var value1); - api.GET(key2, out var value2); + api.GET(key1, out PinnedSpanByte value1); + api.GET(key2, out PinnedSpanByte value2); // Return the two keys as an array of bulk strings WriteBulkStringArray(ref output, value1, value2); diff --git a/main/GarnetServer/Extensions/MGetIfPM.cs b/main/GarnetServer/Extensions/MGetIfPM.cs index e0321376609..2e34e205c07 100644 --- a/main/GarnetServer/Extensions/MGetIfPM.cs +++ b/main/GarnetServer/Extensions/MGetIfPM.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet { @@ -39,11 +40,11 @@ public override void Finalize(TGarnetApi api, ref CustomProcedureInp var prefix = GetNextArg(ref procInput, ref offset); // Read key, check condition, add to output - ArgSlice key; - List values = []; + PinnedSpanByte key; + List values = []; while ((key = GetNextArg(ref procInput, ref offset)).Length > 0) { - if (api.GET(key, out var value) == GarnetStatus.OK) + if (api.GET(key, out PinnedSpanByte value) == GarnetStatus.OK) { if (value.ReadOnlySpan.StartsWith(prefix.ReadOnlySpan)) { diff --git a/main/GarnetServer/Extensions/MSetPx.cs b/main/GarnetServer/Extensions/MSetPx.cs index 7886a437fcf..d547c494fb8 100644 --- a/main/GarnetServer/Extensions/MSetPx.cs +++ b/main/GarnetServer/Extensions/MSetPx.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet { @@ -38,7 +39,7 @@ public override void Finalize(TGarnetApi api, ref CustomProcedureInp var expiryMs = GetNextArg(ref procInput, ref offset); // Read and set key-value pairs with expiry - ArgSlice key, value; + PinnedSpanByte key, value; while ((key = GetNextArg(ref procInput, ref offset)).Length > 0) { value = GetNextArg(ref procInput, ref offset); diff --git a/main/GarnetServer/Extensions/MyDictGet.cs b/main/GarnetServer/Extensions/MyDictGet.cs index 6ae34844acc..9cff0be624b 100644 --- a/main/GarnetServer/Extensions/MyDictGet.cs +++ b/main/GarnetServer/Extensions/MyDictGet.cs @@ -10,7 +10,7 @@ namespace Garnet { public class MyDictGet : CustomObjectFunctions { - public override bool Reader(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) + public override bool Reader(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) { Debug.Assert(value is MyDict); diff --git a/main/GarnetServer/Extensions/MyDictObject.cs b/main/GarnetServer/Extensions/MyDictObject.cs index 44cd15450ef..b82aa95f2ff 100644 --- a/main/GarnetServer/Extensions/MyDictObject.cs +++ b/main/GarnetServer/Extensions/MyDictObject.cs @@ -22,13 +22,13 @@ class MyDict : CustomObjectBase readonly Dictionary dict; public MyDict(byte type) - : base(type, 0, MemoryUtils.DictionaryOverhead) + : base(type, new(MemoryUtils.DictionaryOverhead, sizeof(int))) { dict = new(ByteArrayComparer.Instance); } public MyDict(byte type, BinaryReader reader) - : base(type, reader, MemoryUtils.DictionaryOverhead) + : base(type, reader, new(MemoryUtils.DictionaryOverhead, sizeof(int))) { dict = new(ByteArrayComparer.Instance); @@ -132,9 +132,7 @@ public override unsafe void Scan(long start, out List items, out long cu public bool Set(byte[] key, byte[] value) { if (dict.TryGetValue(key, out var oldValue)) - { UpdateSize(key, oldValue, false); - } dict[key] = value; UpdateSize(key, value); @@ -143,15 +141,25 @@ public bool Set(byte[] key, byte[] value) private void UpdateSize(byte[] key, byte[] value, bool add = true) { - var size = Utility.RoundUp(key.Length, IntPtr.Size) + Utility.RoundUp(value.Length, IntPtr.Size) + var memorySize = Utility.RoundUp(key.Length, IntPtr.Size) + Utility.RoundUp(value.Length, IntPtr.Size) + (2 * MemoryUtils.ByteArrayOverhead) + MemoryUtils.DictionaryEntryOverhead; - this.Size += add ? size : -size; - Debug.Assert(this.Size >= MemoryUtils.DictionaryOverhead); + var kvSize = sizeof(int) * 2 + key.Length + value.Length; + + if (add) + { + this.MemorySize += memorySize; + this.DiskSize += kvSize; + } + else + { + this.MemorySize -= memorySize; + this.DiskSize -= kvSize; + Debug.Assert(this.MemorySize >= MemoryUtils.DictionaryOverhead); + Debug.Assert(this.DiskSize >= sizeof(int)); + } } public bool TryGetValue(byte[] key, [MaybeNullWhen(false)] out byte[] value) - { - return dict.TryGetValue(key, out value); - } + => dict.TryGetValue(key, out value); } } \ No newline at end of file diff --git a/main/GarnetServer/Extensions/MyDictSet.cs b/main/GarnetServer/Extensions/MyDictSet.cs index 60c9ff0e6c2..0a8c14a4c06 100644 --- a/main/GarnetServer/Extensions/MyDictSet.cs +++ b/main/GarnetServer/Extensions/MyDictSet.cs @@ -10,9 +10,9 @@ namespace Garnet { public class MyDictSet : CustomObjectFunctions { - public override bool NeedInitialUpdate(ReadOnlyMemory key, ref ObjectInput input, ref (IMemoryOwner, int) output) => true; + public override bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref (IMemoryOwner, int) output) => true; - public override bool Updater(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) + public override bool Updater(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) { Debug.Assert(value is MyDict); diff --git a/main/GarnetServer/Extensions/ReadWriteTxn.cs b/main/GarnetServer/Extensions/ReadWriteTxn.cs index 57fbeae7ed4..16778b83735 100644 --- a/main/GarnetServer/Extensions/ReadWriteTxn.cs +++ b/main/GarnetServer/Extensions/ReadWriteTxn.cs @@ -21,7 +21,7 @@ sealed class ReadWriteTxn : CustomTransactionProcedure public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { int offset = 0; - api.GET(GetNextArg(ref procInput, ref offset), out var key1); + 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); @@ -36,7 +36,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p var key2 = GetNextArg(ref procInput, ref offset); var key3 = GetNextArg(ref procInput, ref offset); - var status = api.GET(key1, out var result); + var status = api.GET(key1, out PinnedSpanByte result); if (status == GarnetStatus.OK) { api.SET(key2, result); diff --git a/main/GarnetServer/Extensions/Sum.cs b/main/GarnetServer/Extensions/Sum.cs index 1fdf67ba2e0..46baf607db7 100644 --- a/main/GarnetServer/Extensions/Sum.cs +++ b/main/GarnetServer/Extensions/Sum.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet { @@ -12,11 +13,11 @@ public override bool Execute(TGarnetApi garnetApi, ref CustomProcedu { var offset = 0; var sum = 0; - ArgSlice key; + PinnedSpanByte key; while ((key = GetNextArg(ref procInput, ref offset)).Length > 0) { - if (garnetApi.GET(key, out var value) == GarnetStatus.OK) + if (garnetApi.GET(key, out PinnedSpanByte value) == GarnetStatus.OK) { // Sum the values if (int.TryParse(value.ToString(), out var intValue)) diff --git a/metrics/HdrHistogram/Utilities/WriterReaderPhaser.cs b/metrics/HdrHistogram/Utilities/WriterReaderPhaser.cs index 537317ca715..63d4e526b55 100644 --- a/metrics/HdrHistogram/Utilities/WriterReaderPhaser.cs +++ b/metrics/HdrHistogram/Utilities/WriterReaderPhaser.cs @@ -29,12 +29,12 @@ namespace HdrHistogram.Utilities /// "writers" are wait free, "readers" block for other "readers", and "readers" are only blocked by "writers" whose critical was entered before the reader's attempt. /// /// When used to protect an actively recording data structure, the assumptions on how readers and writers act are: - ///
      - ///
    1. There are two sets of data structures("active" and "inactive")
    2. - ///
    3. Writing is done to the perceived active version(as perceived by the writer), and only within critical sections delineated by and ).
    4. - ///
    5. Only readers switch the perceived roles of the active and inactive data structures. - /// They do so only while under , and only before calling .
    6. - ///
    + /// + /// There are two sets of data structures("active" and "inactive") + /// Writing is done to the perceived active version(as perceived by the writer), and only within critical sections delineated by and ). + /// Only readers switch the perceived roles of the active and inactive data structures. + /// They do so only while under , and only before calling . + /// /// When the above assumptions are met, guarantees that the inactive data structures are not being modified by any writers while being read while under protection after a operation. ///
    /// diff --git a/playground/GarnetJSON/JsonCommands.cs b/playground/GarnetJSON/JsonCommands.cs index 7691771af23..6613f8eef6b 100644 --- a/playground/GarnetJSON/JsonCommands.cs +++ b/playground/GarnetJSON/JsonCommands.cs @@ -16,9 +16,9 @@ public class JsonSET : CustomObjectFunctions public JsonSET(ILogger? logger = null) => this.logger = logger; - public override bool NeedInitialUpdate(ReadOnlyMemory key, ref ObjectInput input, ref (IMemoryOwner, int) output) => true; + public override bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref (IMemoryOwner, int) output) => true; - public override bool Updater(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject jsonObject, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) + public override bool Updater(ReadOnlySpan key, ref ObjectInput input, IGarnetObject jsonObject, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) { Debug.Assert(jsonObject is JsonObject); @@ -41,7 +41,7 @@ public class JsonGET : CustomObjectFunctions public JsonGET(ILogger? logger = null) => this.logger = logger; - public override bool Reader(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) + public override bool Reader(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) { Debug.Assert(value is JsonObject); diff --git a/playground/GarnetJSON/JsonObject.cs b/playground/GarnetJSON/JsonObject.cs index f96d1bd6176..a36593a1d81 100644 --- a/playground/GarnetJSON/JsonObject.cs +++ b/playground/GarnetJSON/JsonObject.cs @@ -49,7 +49,7 @@ public class JsonObject : CustomObjectBase ///
    /// The type of the object. public JsonObject(byte type) - : base(type, 0, MemoryUtils.DictionaryOverhead) + : base(type, new(MemoryUtils.DictionaryOverhead, sizeof(int))) { } @@ -89,8 +89,8 @@ public JsonObject(JsonObject obj) /// The binary writer to serialize to. public override void SerializeObject(BinaryWriter writer) { - if (jNode == null) return; - + if (jNode == null) + return; writer.Write(jNode.ToJsonString()); } @@ -150,11 +150,11 @@ public bool TryGet(string path, out string jsonString, ILogger? logger = null) /// The logger to log any errors. /// true if the value was successfully set; otherwise, false. /// Thrown when or is null. + /// TODO: This currently does not update or public bool TrySet(string path, string value, ILogger? logger = null) { if (path == null) throw new ArgumentNullException(nameof(path)); - if (value == null) throw new ArgumentNullException(nameof(value)); @@ -231,9 +231,7 @@ private void Set(string path, string value) // Replace matched value with input value if (match.Value is JsonValue matchValue) - { matchValue.ReplaceWith(valNode); - } } } } @@ -241,9 +239,8 @@ private void Set(string path, string value) private static string GetParentPathExt(string jsonPath) { var matches = Regex.Matches(jsonPath, JsonPathPattern); - - if (matches.Count == 0) return "$"; - + if (matches.Count == 0) + return "$"; return jsonPath.Substring(0, matches[^1].Index); } @@ -261,9 +258,7 @@ private static int GetArrayIndex(string path) { var indexString = path.Substring(startIndex + 1, endIndex - startIndex - 1); if (int.TryParse(indexString, out var index)) - { return index; - } } throw new ArgumentException("Invalid array index in path"); diff --git a/playground/NoOpModule/DummyObject.cs b/playground/NoOpModule/DummyObject.cs index 06cd534d989..d34188bd784 100644 --- a/playground/NoOpModule/DummyObject.cs +++ b/playground/NoOpModule/DummyObject.cs @@ -26,13 +26,13 @@ public class DummyObject : CustomObjectBase { /// public DummyObject(byte type) - : base(type, 0, MemoryUtils.DictionaryOverhead) + : base(type, new(MemoryUtils.DictionaryOverhead, 0)) { } /// public DummyObject(byte type, BinaryReader reader) - : base(type, reader, MemoryUtils.DictionaryOverhead) + : base(type, reader, new(MemoryUtils.DictionaryOverhead, 0)) { } diff --git a/playground/NoOpModule/DummyObjectNoOpRMW.cs b/playground/NoOpModule/DummyObjectNoOpRMW.cs index 8585d166d8c..b1f43697843 100644 --- a/playground/NoOpModule/DummyObjectNoOpRMW.cs +++ b/playground/NoOpModule/DummyObjectNoOpRMW.cs @@ -13,11 +13,11 @@ namespace NoOpModule public class DummyObjectNoOpRMW : CustomObjectFunctions { /// - public override bool NeedInitialUpdate(ReadOnlyMemory key, ref ObjectInput input, + public override bool NeedInitialUpdate(ReadOnlySpan key, ref ObjectInput input, ref (IMemoryOwner, int) output) => true; /// - public override bool Updater(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, + public override bool Updater(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref RMWInfo rmwInfo) { return true; diff --git a/playground/NoOpModule/DummyObjectNoOpRead.cs b/playground/NoOpModule/DummyObjectNoOpRead.cs index 19f2c496637..d2fa50f4c20 100644 --- a/playground/NoOpModule/DummyObjectNoOpRead.cs +++ b/playground/NoOpModule/DummyObjectNoOpRead.cs @@ -13,7 +13,7 @@ namespace NoOpModule public class DummyObjectNoOpRead : CustomObjectFunctions { /// - public override bool Reader(ReadOnlyMemory key, ref ObjectInput input, IGarnetObject value, + public override bool Reader(ReadOnlySpan key, ref ObjectInput input, IGarnetObject value, ref (IMemoryOwner, int) output, ref ReadInfo readInfo) { return true; diff --git a/test/Garnet.test.cluster/ClusterMigrateTests.cs b/test/Garnet.test.cluster/ClusterMigrateTests.cs index 76cb5f06ec5..ca4f8d65aad 100644 --- a/test/Garnet.test.cluster/ClusterMigrateTests.cs +++ b/test/Garnet.test.cluster/ClusterMigrateTests.cs @@ -978,13 +978,13 @@ public void ClusterSimpleMigrateKeys() { resp = context.clusterTestUtils.GetKey(otherNodeIndex, _key, out slot, out endpoint, out responseState, logger: context.logger); } - ClassicAssert.AreEqual(resp, "MOVED"); - ClassicAssert.AreEqual(_workingSlot, slot); - ClassicAssert.AreEqual(context.clusterTestUtils.GetEndPoint(targetNodeIndex), endpoint); + Assert.That(resp, Is.EqualTo("MOVED")); + Assert.That(slot, Is.EqualTo(_workingSlot)); + Assert.That(endpoint, Is.EqualTo(context.clusterTestUtils.GetEndPoint(targetNodeIndex))); resp = context.clusterTestUtils.GetKey(targetNodeIndex, _key, out _, out _, out responseState, logger: context.logger); - ClassicAssert.AreEqual(responseState, ResponseState.OK); - ClassicAssert.AreEqual(resp, _key); + Assert.That(responseState, Is.EqualTo(ResponseState.OK)); + Assert.That(resp, Is.EqualTo(_key)); } context.logger.LogDebug("14. Checking migrate keys done"); diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index 2831ec28f36..2c31c5794d0 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -1750,7 +1750,7 @@ public void MigrateKeys(IPEndPoint source, IPEndPoint target, List keys, { target.Address.ToString(), target.Port, - $"\"\"", + "", 0, -1, "KEYS" diff --git a/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs b/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs index e7a0607cfd2..90a8c7bfd0e 100644 --- a/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs +++ b/test/Garnet.test.cluster/RedirectTests/TestClusterProc.cs @@ -44,11 +44,11 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p var getB = GetNextArg(ref procInput, ref offset); var getC = GetNextArg(ref procInput, ref offset); - var status = api.GET(getA, out _); + var status = api.GET(getA, out PinnedSpanByte _); ClassicAssert.AreEqual(GarnetStatus.NOTFOUND, status); - _ = api.GET(getB, out _); + _ = api.GET(getB, out PinnedSpanByte _); ClassicAssert.AreEqual(GarnetStatus.NOTFOUND, status); - _ = api.GET(getC, out _); + _ = api.GET(getC, out PinnedSpanByte _); ClassicAssert.AreEqual(GarnetStatus.NOTFOUND, status); WriteSimpleString(ref output, "SUCCESS"); } @@ -95,7 +95,6 @@ sealed class TestClusterReadWriteCustomTxn : CustomTransactionProcedure /// CLUSTERSETPROC key1 key2 key3 ///
    /// - /// /// public override bool Prepare(TGarnetReadApi api, ref CustomProcedureInput procInput) { @@ -115,13 +114,13 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p { var offset = 0; var getA = GetNextArg(ref procInput, ref offset); - var setB = GetNextArg(ref procInput, ref offset).SpanByte; - var setC = GetNextArg(ref procInput, ref offset).SpanByte; + var setB = GetNextArg(ref procInput, ref offset); + var setC = GetNextArg(ref procInput, ref offset); - _ = api.GET(getA, out _); - var status = api.SET(ref setB, ref setB); + _ = api.GET(getA, out PinnedSpanByte _); + var status = api.SET(setB, setB); ClassicAssert.AreEqual(GarnetStatus.OK, status); - status = api.SET(ref setC, ref setC); + status = api.SET(setC, setC); ClassicAssert.AreEqual(GarnetStatus.OK, status); WriteSimpleString(ref output, "SUCCESS"); } diff --git a/test/Garnet.test/CacheSizeTrackerTests.cs b/test/Garnet.test/CacheSizeTrackerTests.cs index 93fe4430c56..fe2ee5780d5 100644 --- a/test/Garnet.test/CacheSizeTrackerTests.cs +++ b/test/Garnet.test/CacheSizeTrackerTests.cs @@ -11,14 +11,14 @@ namespace Garnet.test { - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; [TestFixture] public class CacheSizeTrackerTests { GarnetServer server; - TsavoriteKV objStore; + TsavoriteKV objStore; CacheSizeTracker cacheSizeTracker; [SetUp] @@ -55,7 +55,7 @@ public void HeapSizeValidationTest() [Test, CancelAfter(40 * 1000)] public void IncreaseEmptyPageCountTest() { - ManualResetEventSlim epcEvent = new ManualResetEventSlim(false); + var epcEvent = new ManualResetEventSlim(false); int emptyPageCountIncrements = 0; cacheSizeTracker.mainLogTracker.PostEmptyPageCountIncrease = (int count) => { emptyPageCountIncrements++; if (emptyPageCountIncrements == 3) epcEvent.Set(); }; @@ -79,14 +79,9 @@ public void IncreaseEmptyPageCountTest() ClassicAssert.AreEqual(5952, cacheSizeTracker.mainLogTracker.LogHeapSizeBytes); // 24 * 248 for each hashset object - // Wait for the resizing to happen - bool eventSignaled = epcEvent.Wait( - TimeSpan.FromSeconds(3 * LogSizeTracker.resizeTaskDelaySeconds)); // Wait for 3x resize task delay - - if (!eventSignaled) - { + // Wait for 3x resize task delay for the resizing to happen + if (!epcEvent.Wait(TimeSpan.FromSeconds(3 * LogSizeTracker.resizeTaskDelaySeconds))) Assert.Fail("Timeout occurred. Resizing did not happen within the specified time."); - } } [Test] @@ -125,7 +120,7 @@ public void ReadCacheIncreaseEmptyPageCountTest() var info = TestUtils.GetStoreAddressInfo(redis.GetServer(TestUtils.EndPoint), includeReadCache: true, isObjectStore: true); ClassicAssert.AreEqual(632, info.ReadCacheTailAddress); // 25 (records) * 24 (rec size) + 24 (initial) + 8 (page boundary) - 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/Extensions/ProcCustomCmd.cs b/test/Garnet.test/Extensions/ProcCustomCmd.cs index 1fc59947269..7a807b0c270 100644 --- a/test/Garnet.test/Extensions/ProcCustomCmd.cs +++ b/test/Garnet.test/Extensions/ProcCustomCmd.cs @@ -3,6 +3,7 @@ using Garnet.common; using Garnet.server; +using Tsavorite.core; namespace Garnet { @@ -13,7 +14,7 @@ public override unsafe bool Execute(TGarnetApi garnetApi, ref Custom var offset = 0; var key = GetNextArg(ref procInput, ref offset); - var args = new ArgSlice[2]; + var args = new PinnedSpanByte[2]; args[0] = GetNextArg(ref procInput, ref offset); // value to set args[1] = GetNextArg(ref procInput, ref offset); // prefix to match diff --git a/test/Garnet.test/Extensions/RateLimiterTxn.cs b/test/Garnet.test/Extensions/RateLimiterTxn.cs index e1fdc0816b9..f9b45ac9ee1 100644 --- a/test/Garnet.test/Extensions/RateLimiterTxn.cs +++ b/test/Garnet.test/Extensions/RateLimiterTxn.cs @@ -53,7 +53,7 @@ public override unsafe void Main(TGarnetApi api, ref CustomProcedure var timeInMicroSecondBytes = Encoding.ASCII.GetBytes(timeInMicroSecond.ToString()); fixed (byte* timeInMicroSecondBytesPtr = timeInMicroSecondBytes) { - api.SortedSetAdd(key, new ArgSlice(unixTimeInMilliSecondPtr, unixTimeInMilliSecondBytes.Length), new ArgSlice(timeInMicroSecondBytesPtr, timeInMicroSecondBytes.Length), out var _); + 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); } } diff --git a/test/Garnet.test/Extensions/TxnCustomCmd.cs b/test/Garnet.test/Extensions/TxnCustomCmd.cs index 989852c2ba5..48646f46f7d 100644 --- a/test/Garnet.test/Extensions/TxnCustomCmd.cs +++ b/test/Garnet.test/Extensions/TxnCustomCmd.cs @@ -42,7 +42,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p var myDictField = GetNextArg(ref procInput, ref offset); var myDictValue = GetNextArg(ref procInput, ref offset); - var args = new ArgSlice[2]; + var args = new PinnedSpanByte[2]; args[0] = myDictField; args[1] = myDictValue; diff --git a/test/Garnet.test/GarnetBitmapTests.cs b/test/Garnet.test/GarnetBitmapTests.cs index a7202baf61a..d1f71a29f8c 100644 --- a/test/Garnet.test/GarnetBitmapTests.cs +++ b/test/Garnet.test/GarnetBitmapTests.cs @@ -15,7 +15,7 @@ namespace Garnet.test public class GarnetBitmapTests { GarnetServer server; - Random r; + Random rng; [SetUp] public void Setup() @@ -23,7 +23,7 @@ public void Setup() TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir); server.Start(); - r = new Random(674386); + rng = new Random(674386); } [TearDown] @@ -33,12 +33,12 @@ public void TearDown() TestUtils.DeleteDirectory(TestUtils.MethodTestDir); } - private long LongRandom() => ((long)this.r.Next() << 32) | (long)this.r.Next(); + private long LongRandom() => ((long)this.rng.Next() << 32) | (long)this.rng.Next(); private ulong ULongRandom() { - ulong lsb = (ulong)(this.r.Next()); - ulong msb = (ulong)(this.r.Next()) << 32; + ulong lsb = (ulong)(this.rng.Next()); + ulong msb = (ulong)(this.rng.Next()) << 32; return (msb | lsb); } @@ -215,8 +215,8 @@ public void BitmapSetGetBitTest_LTM(bool preSet) for (int j = 0; j < keyIter; j++) { - long offset = r.Next(0, bitmapBytes << 3); - bool set = r.Next(0, 1) == 0 ? false : true; + long offset = rng.Next(0, bitmapBytes << 3); + bool set = rng.Next(0, 1) == 0 ? false : true; bool returnedVal = db.StringSetBit(sKey, offset, set); bool expectedVal = false; @@ -248,7 +248,7 @@ public void BitmapSetGetBitTest_LTM(bool preSet) for (int j = 0; j < keyIter; j++) { - long offset = r.Next(0, bitmapBytes << 3); + long offset = rng.Next(0, bitmapBytes << 3); bool returnedVal = db.StringGetBit(sKey, offset); bool expectedVal = false; if (state.ContainsKey(key) && state[key].ContainsKey(offset)) @@ -306,7 +306,7 @@ void SimpleBitCountTest() for (var i = 0; i < iter; i++) { - var offset = r.Next(1, maxBitmapLen); + var offset = rng.Next(1, maxBitmapLen); var set = !db.StringSetBit(key, offset, true); expectedCount += set ? 1 : 0; } @@ -370,7 +370,7 @@ public void BitmapBitCountBetweenOffsetsTest() long maxOffset = 0; for (int i = 0; i < iter; i++) { - long offset = r.Next(1, maxBitmapLen); + long offset = rng.Next(1, maxBitmapLen); db.StringSetBit(key, offset, true); maxOffset = Math.Max(offset, maxOffset); offsets.Add(offset); @@ -391,12 +391,12 @@ public void BitmapBitCountBetweenOffsetsTest() long expectedCount = Count(bitmap, 0, -1); count = db.StringBitCount(key, 0, -1); - ClassicAssert.AreEqual(count, expectedCount, $"{0} {-1} {bitmap.Length}"); + ClassicAssert.AreEqual(expectedCount, count, $"{0} {-1} {bitmap.Length}"); //Test with startOffset for (int i = 0; i < iter; i++) { - int startOffset = r.Next(1, (int)maxSizeInBytes); + int startOffset = rng.Next(1, (int)maxSizeInBytes); expectedCount = Count(bitmap, startOffset, -1); count = db.StringBitCount(key, startOffset); @@ -406,8 +406,8 @@ public void BitmapBitCountBetweenOffsetsTest() //Test with startOffset and endOffset for (int i = 0; i < iter; i++) { - int startOffset = r.Next(1, (int)maxSizeInBytes); - int endOffset = r.Next(startOffset, (int)maxSizeInBytes); + int startOffset = rng.Next(1, (int)maxSizeInBytes); + int endOffset = rng.Next(startOffset, (int)maxSizeInBytes); expectedCount = Count(bitmap, startOffset, endOffset); count = db.StringBitCount(key, startOffset, endOffset); @@ -434,12 +434,12 @@ public void BitmapBitCountBetweenOffsetsTestV2() for (int j = 0; j < iter; j++) { for (int i = 0; i < buf.Length; i++) - buf[i] = (byte)r.Next(0, 128); + buf[i] = (byte)rng.Next(0, 128); db.StringSet(key, buf); - int startOffset = r.Next(1, buf.Length); - int endOffset = r.Next(startOffset, buf.Length); + int startOffset = rng.Next(1, buf.Length); + int endOffset = rng.Next(startOffset, buf.Length); long expectedCount = Count(buf, startOffset, endOffset); count = db.StringBitCount(key, startOffset, endOffset); @@ -466,11 +466,11 @@ public void BitmapBitCountNegativeOffsets() //check offsets in range for (int j = 0; j < iter; j++) { - r.NextBytes(buf); + rng.NextBytes(buf); db.StringSet(key, buf); - int startOffset = j == 0 ? -10 : r.Next(-maxByteLen, 0); - int endOffset = j == 0 ? -1 : r.Next(startOffset, 0); + int startOffset = j == 0 ? -10 : rng.Next(-maxByteLen, 0); + int endOffset = j == 0 ? -1 : rng.Next(startOffset, 0); expectedCount = Count(buf, startOffset, endOffset); count = db.StringBitCount(key, startOffset, endOffset); @@ -481,11 +481,11 @@ public void BitmapBitCountNegativeOffsets() //check negative offsets beyond range for (int j = 0; j < iter; j++) { - r.NextBytes(buf); + rng.NextBytes(buf); db.StringSet(key, buf); - int startOffset = j == 0 ? -10 : r.Next(-maxByteLen << 1, -maxByteLen); - int endOffset = j == 0 ? -1 : r.Next(startOffset, -maxByteLen); + int startOffset = j == 0 ? -10 : rng.Next(-maxByteLen << 1, -maxByteLen); + int endOffset = j == 0 ? -1 : rng.Next(startOffset, -maxByteLen); expectedCount = Count(buf, startOffset, endOffset); count = db.StringBitCount(key, startOffset, endOffset); @@ -515,7 +515,7 @@ public void BitmapBitCountTest_LTM() for (int i = 0; i < keyCount; i++) { string sKey = i.ToString(); - r.NextBytes(bitmap); + rng.NextBytes(bitmap); bitmapList.Add(Count(bitmap)); db.StringSet(sKey, bitmap); @@ -524,7 +524,7 @@ public void BitmapBitCountTest_LTM() int iter = 128; for (int i = 0; i < iter; i++) { - int key = r.Next(0, keyCount); + int key = rng.Next(0, keyCount); string sKey = key.ToString(); long count = db.StringBitCount(sKey); long expectedCount = bitmapList[key]; @@ -546,7 +546,7 @@ public unsafe void BitmapSimpleBITCOUNT_PCT(int bytesPerSend) string key = "mykey"; int maxBitmapLen = 1 << 12; byte[] buf = new byte[maxBitmapLen >> 3]; - r.NextBytes(buf); + rng.NextBytes(buf); db.StringSet(key, buf); long expectedCount = Count(buf); @@ -611,7 +611,7 @@ public void BitmapSimpleBitPosTests() long maxOffset = 0; for (var i = 0; i < iter; i++) { - long offset = r.Next(1, maxBitmapLen); + long offset = rng.Next(1, maxBitmapLen); _ = db.StringSetBit(key, offset, true); buf = db.StringGet(key); @@ -634,7 +634,7 @@ public void BitmapSimpleBitPosTests() for (var i = 0; i < iter; i++) { - long offset = r.Next(1, (int)maxOffset); + long offset = rng.Next(1, (int)maxOffset); _ = db.StringSetBit(key, offset, false); buf = db.StringGet(key); @@ -667,13 +667,13 @@ public void BitmapBitPosOffsetsTest() for (var j = 0; j < iter; j++) { - r.NextBytes(buf); + rng.NextBytes(buf); _ = db.StringSet(key, buf); - var startOffset = r.Next(0, maxByteLen); - var endOffset = r.Next(startOffset, maxByteLen); + var startOffset = rng.Next(0, maxByteLen); + var endOffset = rng.Next(startOffset, maxByteLen); - var set = r.Next(0, 1) == 0 ? false : true; + var set = rng.Next(0, 1) == 0 ? false : true; expectedPos = Bitpos(buf, startOffset, endOffset, set); pos = db.StringBitPosition(key, set, startOffset, endOffset); @@ -688,13 +688,13 @@ public void BitmapBitPosOffsetsTest() // check negative offsets in range for (var j = 0; j < iter; j++) { - r.NextBytes(buf); + rng.NextBytes(buf); _ = db.StringSet(key, buf); - var startOffset = j == 0 ? -10 : r.Next(-maxByteLen, 0); - var endOffset = j == 0 ? -1 : r.Next(startOffset, 0); + int startOffset = j == 0 ? -10 : rng.Next(-maxByteLen, 0); + var endOffset = j == 0 ? -1 : rng.Next(startOffset, 0); - var set = r.Next(0, 1) != 0; + var set = rng.Next(0, 1) != 0; expectedPos = Bitpos(buf, startOffset, endOffset, set); pos = db.StringBitPosition(key, set, startOffset, endOffset); ClassicAssert.AreEqual(expectedPos, pos, $"{j} {set} {startOffset} {endOffset}"); @@ -726,8 +726,8 @@ public void BitmapBitPosTest_LTM() for (var i = 0; i < keyCount; i++) { - var sKey = i.ToString(); - r.NextBytes(bitmap); + string sKey = i.ToString(); + rng.NextBytes(bitmap); bitmapList.Add(Bitpos(bitmap, set: true)); _ = db.StringSet(sKey, bitmap); @@ -736,7 +736,7 @@ public void BitmapBitPosTest_LTM() var iter = 128; for (var i = 0; i < iter; i++) { - var key = r.Next(0, keyCount); + int key = rng.Next(0, keyCount); var sKey = key.ToString(); var pos = db.StringBitPosition(sKey, true); var expectedPos = bitmapList[key]; @@ -777,10 +777,10 @@ public unsafe void BitmapSimpleBITPOS_PCT(int bytesPerSend) using var lightClientRequest = TestUtils.CreateRequest(); var db = redis.GetDatabase(0); - var key = "mykey"; + string key = "mykey"; var maxBitmapLen = 1 << 12; var buf = new byte[maxBitmapLen >> 3]; - r.NextBytes(buf); + rng.NextBytes(buf); db.StringSet(key, buf); var expectedPos = Bitpos(buf); @@ -999,20 +999,20 @@ public void BitmapSimpleVarLenBitOpTests() Bitwise[] bitwiseOps = [Bitwise.And, Bitwise.Or, Bitwise.Xor, Bitwise.And, Bitwise.Or, Bitwise.Xor]; int maxBytes = 512; - byte[] dataA = new byte[r.Next(1, maxBytes)]; - byte[] dataB = new byte[r.Next(1, maxBytes)]; - byte[] dataC = new byte[r.Next(1, maxBytes)]; - byte[] dataD = new byte[r.Next(1, maxBytes)]; + byte[] dataA = new byte[rng.Next(1, maxBytes)]; + byte[] dataB = new byte[rng.Next(1, maxBytes)]; + byte[] dataC = new byte[rng.Next(1, maxBytes)]; + byte[] dataD = new byte[rng.Next(1, maxBytes)]; byte[] dataX = null; for (int j = 0; j < bitwiseOps.Length; j++) { for (int i = 0; i < tests; i++) { - r.NextBytes(dataA); - r.NextBytes(dataB); - r.NextBytes(dataC); - r.NextBytes(dataD); + rng.NextBytes(dataA); + rng.NextBytes(dataB); + rng.NextBytes(dataC); + rng.NextBytes(dataD); db.StringSet(a, dataA); db.StringSet(b, dataB); @@ -1039,12 +1039,11 @@ public void BitmapSimpleVarLenBitOpTests() ApplyBitop(ref dataX, dataC, f8); ApplyBitop(ref dataX, dataD, f8); - long size = db.StringBitOperation(bitwiseOps[j], x, keys); - ClassicAssert.AreEqual(size, dataX.Length); - - byte[] expectedX = db.StringGet(x); + long actualSize = db.StringBitOperation(bitwiseOps[j], x, keys); + ClassicAssert.AreEqual(dataX.Length, actualSize); - ClassicAssert.AreEqual(dataX, expectedX); + byte[] actualX = db.StringGet(x); + ClassicAssert.AreEqual(dataX, actualX); } } } @@ -1070,11 +1069,11 @@ public void BitmapBitOpNotTest() string dstKey = "dstKey"; int maxBytes = 256; - byte[] srcVal = new byte[r.Next(1, maxBytes)]; + byte[] srcVal = new byte[rng.Next(1, maxBytes)]; byte[] dstVal; for (int i = 0; i < tests; i++) { - r.NextBytes(srcVal); + rng.NextBytes(srcVal); db.StringSet(srcKey, srcVal); dstVal = db.StringGet(srcKey); @@ -1113,8 +1112,8 @@ public void BitmapSimpleBitOpVarLenGrowingSizeTests() //Test NOT for (int i = 0; i < tests; i++) { - dataA = new byte[r.Next(minSize, minSize + 32)]; - r.NextBytes(dataA); + dataA = new byte[rng.Next(minSize, minSize + 32)]; + rng.NextBytes(dataA); db.StringSet(a, dataA); dataX = null; @@ -1131,16 +1130,16 @@ public void BitmapSimpleBitOpVarLenGrowingSizeTests() { for (int i = 0; i < tests; i++) { - dataA = new byte[r.Next(minSize, minSize + 16)]; minSize = dataA.Length; - dataB = new byte[r.Next(minSize, minSize + 16)]; minSize = dataB.Length; - dataC = new byte[r.Next(minSize, minSize + 16)]; minSize = dataC.Length; - dataD = new byte[r.Next(minSize, minSize + 16)]; minSize = dataD.Length; + dataA = new byte[rng.Next(minSize, minSize + 16)]; minSize = dataA.Length; + dataB = new byte[rng.Next(minSize, minSize + 16)]; minSize = dataB.Length; + dataC = new byte[rng.Next(minSize, minSize + 16)]; minSize = dataC.Length; + dataD = new byte[rng.Next(minSize, minSize + 16)]; minSize = dataD.Length; minSize = 17; - r.NextBytes(dataA); - r.NextBytes(dataB); - r.NextBytes(dataC); - r.NextBytes(dataD); + rng.NextBytes(dataA); + rng.NextBytes(dataB); + rng.NextBytes(dataC); + rng.NextBytes(dataD); db.StringSet(a, dataA); db.StringSet(b, dataB); @@ -1264,10 +1263,10 @@ public void BitmapBitfieldGetTest([Values(RespCommand.BITFIELD, RespCommand.BITF long expectedValue; long returnedValue; long redisValue; - r = new Random(Guid.NewGuid().GetHashCode()); + rng = new Random(Guid.NewGuid().GetHashCode()); bitmapData = new byte[16]; - r.NextBytes(bitmapData); + rng.NextBytes(bitmapData); db.StringSet(key, bitmapData); for (int i = 0; i < (bitmapData.Length << 3) + 64; i++)//offset in bits { @@ -1321,7 +1320,7 @@ public unsafe void BitmapBitfieldGetTest_PCT([Values(RespCommand.BITFIELD, RespC //r = new Random(Guid.NewGuid().GetHashCode()); bitmapData = new byte[16]; - r.NextBytes(bitmapData); + rng.NextBytes(bitmapData); db.StringSet(key, bitmapData); for (int i = 0; i < (bitmapData.Length << 3) + 64; i++)//offset in bits { @@ -1376,7 +1375,7 @@ public void BitmapBitfieldGetTest_LTM([Values(RespCommand.BITFIELD, RespCommand. for (int i = 0; i < keyCount; i++) { bitmapData[i] = new byte[bitmapBytes]; - r.NextBytes(bitmapData[i]); + rng.NextBytes(bitmapData[i]); int key = i; string sKey = i.ToString(); @@ -1386,11 +1385,11 @@ public void BitmapBitfieldGetTest_LTM([Values(RespCommand.BITFIELD, RespCommand. int iter = 1 << 12; for (int i = 0; i < iter; i++) { - int key = r.Next(0, keyCount); + int key = rng.Next(0, keyCount); byte[] currBitmap = bitmapData[key]; string sKey = key.ToString(); - int offset = r.Next(0, (bitmapData.Length << 3)); - int bitCount = r.Next(1, 65); + int offset = rng.Next(0, (bitmapData.Length << 3)); + int bitCount = rng.Next(1, 65); //signed expectedValue = GetValueFromBitmap(ref currBitmap, offset, bitCount, true); @@ -1416,7 +1415,7 @@ private long RandomIntBitRange(int bitCount, bool signed) long value = LongRandom(); - value = (r.Next() & 0x1) == 0x1 ? -value : value; + value = (rng.Next() & 0x1) == 0x1 ? -value : value; value = value >> (64 - bitCount); ClassicAssert.IsTrue(value >= minVal); @@ -1477,7 +1476,7 @@ public unsafe void BitmapBitfieldSetTest_PCT(int bytesPerSend) //r = new Random(Guid.NewGuid().GetHashCode()); bitmapData = new byte[16]; - r.NextBytes(bitmapData); + rng.NextBytes(bitmapData); db.StringSet(key, bitmapData); long oldVal, expectedOldVal; @@ -1486,8 +1485,8 @@ public unsafe void BitmapBitfieldSetTest_PCT(int bytesPerSend) //1. Test signed set bitfield for (int i = 0; i < tests; i++) { - int bitCount = r.Next(1, 64); - long offset = r.Next(0, (bitmapData.Length << 3) - bitCount - 1); + int bitCount = rng.Next(1, 64); + long offset = rng.Next(0, (bitmapData.Length << 3) - bitCount - 1); //expectedReturnVal = RandomIntBitRange(bitCount); expectedReturnVal = RandomIntBitRange(bitCount, true); @@ -1525,7 +1524,7 @@ public void BitmapBitfieldSetTest() //r = new Random(Guid.NewGuid().GetHashCode()); bitmapData = new byte[16]; - r.NextBytes(bitmapData); + rng.NextBytes(bitmapData); db.StringSet(key, bitmapData); long oldVal, expectedOldVal; @@ -1534,8 +1533,8 @@ public void BitmapBitfieldSetTest() //1. Test signed set bitfield for (int i = 0; i < tests; i++) { - int bitCount = r.Next(1, 64); - long offset = r.Next(0, (bitmapData.Length << 3) - bitCount - 1); + int bitCount = rng.Next(1, 64); + long offset = rng.Next(0, (bitmapData.Length << 3) - bitCount - 1); //expectedReturnVal = RandomIntBitRange(bitCount); expectedReturnVal = RandomIntBitRange(bitCount, true); @@ -1575,7 +1574,7 @@ public void BitmapBitfieldSetTest_LTM() for (int i = 0; i < keyCount; i++) { bitmapData[i] = new byte[bitmapBytes]; - r.NextBytes(bitmapData[i]); + rng.NextBytes(bitmapData[i]); int key = i; string sKey = i.ToString(); @@ -1590,11 +1589,11 @@ public void BitmapBitfieldSetTest_LTM() int iter = 1 << 12; for (int i = 0; i < iter; i++) { - int key = r.Next(0, keyCount); + int key = rng.Next(0, keyCount); byte[] currBitmap = bitmapData[key]; string sKey = key.ToString(); - int offset = r.Next(0, (bitmapData.Length << 3)); - int bitCount = r.Next(1, 65); + int offset = rng.Next(0, (bitmapData.Length << 3)); + int bitCount = rng.Next(1, 65); setNewValue = RandomIntBitRange(bitCount, true); @@ -1731,7 +1730,7 @@ public unsafe void BitmapBitfieldSignedIncrTest_PCT(int bytesPerSend) int testCheckOverflow = 1 << 15; for (int i = 0; i < testCheckOverflow; i++) { - bitCount = r.Next(1, 64); + bitCount = rng.Next(1, 64); long value = RandomIntBitRange(bitCount, true); long incrBy = RandomIntBitRange(bitCount, true); @@ -1811,7 +1810,7 @@ public unsafe void BitmapBitfieldSignedIncrTest_PCT(int bytesPerSend) //signed overflow with wrap and sat for (int i = 0; i < tests; i++) { - bitCount = r.Next(1, 64); + bitCount = rng.Next(1, 64); long value = RandomIntBitRange(bitCount, true); long incrBy = RandomIntBitRange(bitCount, true); @@ -1901,7 +1900,7 @@ public void BitmapBitfieldSignedIncrTest() int testCheckOverflow = 1 << 15; for (int i = 0; i < testCheckOverflow; i++) { - bitCount = r.Next(1, 64); + bitCount = rng.Next(1, 64); long value = RandomIntBitRange(bitCount, true); long incrBy = RandomIntBitRange(bitCount, true); @@ -1970,7 +1969,7 @@ public void BitmapBitfieldSignedIncrTest() //signed overflow with wrap and sat for (int i = 0; i < tests; i++) { - bitCount = r.Next(1, 64); + bitCount = rng.Next(1, 64); long value = RandomIntBitRange(bitCount, true); long incrBy = RandomIntBitRange(bitCount, true); @@ -2041,7 +2040,7 @@ public void BitmapBitfieldIncrTest_LTM() for (int i = 0; i < keyCount; i++) { bitmapData[i] = new byte[bitmapBytes]; - r.NextBytes(bitmapData[i]); + rng.NextBytes(bitmapData[i]); int key = i; string sKey = i.ToString(); @@ -2057,11 +2056,11 @@ public void BitmapBitfieldIncrTest_LTM() int iter = 1 << 12; for (int i = 0; i < iter; i++) { - int key = r.Next(0, keyCount); + int key = rng.Next(0, keyCount); byte[] currBitmap = bitmapData[key]; string sKey = key.ToString(); - int offset = r.Next(0, (bitmapData.Length << 3)); - int bitCount = r.Next(1, 65); + int offset = rng.Next(0, (bitmapData.Length << 3)); + int bitCount = rng.Next(1, 65); setNewValue = RandomIntBitRange(bitCount, true); incrByValue = RandomIntBitRange(bitCount, true); @@ -2129,7 +2128,7 @@ public void BitmapBitfieldUnsignedIncrTest() for (int i = 0; i < tests; i++) { - bitCount = r.Next(1, 63); + bitCount = rng.Next(1, 63); long value = RandomIntBitRange(bitCount, false); long incrBy = RandomIntBitRange(bitCount, true); @@ -2213,11 +2212,11 @@ public void BitmapBitfieldGrowingTest() long incrBy = RandomIntBitRange(bitCount, true); result = (long)db.Execute("BITFIELD", (RedisKey)key, "OVERFLOW", "WRAP", "INCRBY", "i" + bitCount.ToString(), "#" + offset.ToString(), value); - ClassicAssert.AreEqual(result, value); + ClassicAssert.AreEqual(value, result); result = (long)db.Execute("BITFIELD", (RedisKey)key, "OVERFLOW", "WRAP", "INCRBY", "i" + bitCount.ToString(), "#" + offset.ToString(), incrBy); (expectedResult, overflow) = CheckSignedBitfieldOverflow(value, incrBy, (byte)bitCount, 0); - ClassicAssert.AreEqual(result, expectedResult); + ClassicAssert.AreEqual(expectedResult, result); } //sat incrby @@ -2531,7 +2530,7 @@ public void BitmapBitPosBitSearchSingleBitRangeTests() var valueLenBits = valueLen << 3; for (var i = 0; i < iter; i++) { - var offset = r.NextInt64(0, valueLenBits); + var offset = rng.NextInt64(0, valueLenBits); BitSearch(offset, searchFor: true); BitSearch(offset, searchFor: false); } diff --git a/test/Garnet.test/GarnetObjectTests.cs b/test/Garnet.test/GarnetObjectTests.cs index c507aef5b81..c9dff9efd6f 100644 --- a/test/Garnet.test/GarnetObjectTests.cs +++ b/test/Garnet.test/GarnetObjectTests.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Threading.Tasks; using Garnet.server; using NUnit.Framework; @@ -9,13 +10,13 @@ namespace Garnet.test { - using ObjectStoreAllocator = GenericAllocator>>; - using ObjectStoreFunctions = StoreFunctions>; + using ObjectStoreAllocator = ObjectAllocator>; + using ObjectStoreFunctions = StoreFunctions; [TestFixture] public class GarnetObjectTests { - TsavoriteKV store; + TsavoriteKV store; IDevice logDevice, objectLogDevice; [SetUp] @@ -38,113 +39,124 @@ public void TearDown() [Test] public void WriteRead() { - using var session = store.NewSession>(new SimpleSessionFunctions()); + using var session = store.NewSession(new SimpleGarnetObjectSessionFunctions()); var bContext = session.BasicContext; - var key = new byte[] { 0 }; + var key = new ReadOnlySpan([0]); var obj = new SortedSetObject(); - bContext.Upsert(key, obj); + _ = bContext.Upsert(key, obj); IGarnetObject output = null; - var status = bContext.Read(ref key, ref output); + var status = bContext.Read(key, ref output); ClassicAssert.IsTrue(status.Found); ClassicAssert.AreEqual(obj, output); } + const int keyNum = 0; + [Test] public async Task WriteCheckpointRead() { - var session = store.NewSession(new MyFunctions()); - var bContext = session.BasicContext; - - var key = new byte[] { 0 }; var obj = new SortedSetObject(); - obj.Add([15], 10); - - bContext.Upsert(key, obj); - - session.Dispose(); - - await store.TakeHybridLogCheckpointAsync(CheckpointType.FoldOver); + LocalWrite(); + _ = await store.TakeHybridLogCheckpointAsync(CheckpointType.FoldOver); store.Dispose(); CreateStore(); + _ = store.Recover(); + LocalRead(); - store.Recover(); + void LocalWrite() + { + using var session = store.NewSession(new MyFunctions()); + var bContext = session.BasicContext; - session = store.NewSession(new MyFunctions()); - bContext = session.BasicContext; + var key = new ReadOnlySpan([keyNum]); + obj.Add([15], 10); - IGarnetObject output = null; - var status = bContext.Read(ref key, ref output); + _ = bContext.Upsert(key, obj); + } + + void LocalRead() + { + using var session = store.NewSession(new MyFunctions()); + var bContext = session.BasicContext; - session.Dispose(); + IGarnetObject output = null; + var key = new ReadOnlySpan([keyNum]); + var status = bContext.Read(key, ref output); - ClassicAssert.IsTrue(status.Found); - ClassicAssert.IsTrue(obj.Equals((SortedSetObject)output)); + ClassicAssert.IsTrue(status.Found); + ClassicAssert.IsTrue(obj.Equals((SortedSetObject)output)); + } } [Test] public async Task CopyUpdate() { - var session = store.NewSession(new MyFunctions()); - var bContext = session.BasicContext; - - var key = new byte[] { 0 }; IGarnetObject obj = new SortedSetObject(); - ((SortedSetObject)obj).Add([15], 10); - - bContext.Upsert(key, obj); - - store.Log.Flush(true); - - bContext.RMW(ref key, ref obj); - - session.Dispose(); - - await store.TakeHybridLogCheckpointAsync(CheckpointType.FoldOver); + LocalWrite(); + _ = await store.TakeHybridLogCheckpointAsync(CheckpointType.FoldOver); store.Dispose(); CreateStore(); + _ = store.Recover(); + LocalRead(); - store.Recover(); + void LocalWrite() + { + using var session = store.NewSession(new MyFunctions()); + var bContext = session.BasicContext; - session = store.NewSession(new MyFunctions()); - bContext = session.BasicContext; + var key = new ReadOnlySpan([keyNum]); + ((SortedSetObject)obj).Add([15], 10); - IGarnetObject output = null; - var status = bContext.Read(ref key, ref output); + _ = bContext.Upsert(key, obj); + store.Log.Flush(true); + _ = bContext.RMW(key, ref obj); + } - session.Dispose(); + void LocalRead() + { + using var session = store.NewSession(new MyFunctions()); + var bContext = session.BasicContext; - ClassicAssert.IsTrue(status.Found); - ClassicAssert.IsTrue(((SortedSetObject)obj).Equals((SortedSetObject)output)); + IGarnetObject output = null; + var key = new ReadOnlySpan([keyNum]); + var status = bContext.Read(key, ref output); + + ClassicAssert.IsTrue(status.Found); + ClassicAssert.IsTrue(((SortedSetObject)obj).Equals((SortedSetObject)output)); + } } - private class MyFunctions : SessionFunctionsBase + private class MyFunctions : SessionFunctionsBase { public MyFunctions() { } - public override bool SingleReader(ref byte[] key, ref IGarnetObject input, ref IGarnetObject value, ref IGarnetObject dst, ref ReadInfo updateInfo) + public override bool Reader(ref TSourceLogRecord srcLogRecord, ref IGarnetObject input, ref IGarnetObject output, ref ReadInfo readInfo) { - dst = value; + output = (IGarnetObject)srcLogRecord.ValueObject; return true; } - public override bool ConcurrentReader(ref byte[] key, ref IGarnetObject input, ref IGarnetObject value, ref IGarnetObject dst, ref ReadInfo updateInfo, ref RecordInfo recordInfo) + public override bool CopyUpdater(ref TSourceLogRecord srcLogRecord, ref LogRecord dstLogRecord, ref RecordSizeInfo sizeInfo, ref IGarnetObject input, ref IGarnetObject output, ref RMWInfo rmwInfo) { - dst = value; + _ = ((IGarnetObject)srcLogRecord.ValueObject).CopyUpdate(srcLogRecord.Info.IsInNewVersion, ref rmwInfo); return true; } - public override bool CopyUpdater(ref byte[] key, ref IGarnetObject input, ref IGarnetObject oldValue, ref IGarnetObject newValue, ref IGarnetObject output, ref RMWInfo rmwInfo, ref RecordInfo recordInfo) - { - oldValue.CopyUpdate(ref oldValue, ref newValue, false); - return true; - } + public override unsafe RecordFieldInfo GetRMWModifiedFieldInfo(ref TSourceLogRecord srcLogRecord, ref IGarnetObject input) + => new() { KeyDataSize = srcLogRecord.Key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; + public override unsafe RecordFieldInfo GetRMWInitialFieldInfo(ReadOnlySpan key, ref IGarnetObject input) + => new() { KeyDataSize = key.Length, ValueDataSize = 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 }; + public override unsafe RecordFieldInfo GetUpsertFieldInfo(ReadOnlySpan key, IHeapObject value, ref IGarnetObject input) + => new() { KeyDataSize = key.Length, ValueDataSize = ObjectIdMap.ObjectIdSize, ValueIsObject = true }; } private void CreateStore() @@ -152,7 +164,7 @@ private void CreateStore() logDevice ??= Devices.CreateLogDevice(TestUtils.MethodTestDir + "/hlog.log"); objectLogDevice ??= Devices.CreateLogDevice(TestUtils.MethodTestDir + "/hlog.obj.log"); - var kvSettings = new KVSettings + var kvSettings = new KVSettings { IndexSize = 1L << 13, LogDevice = logDevice, @@ -161,7 +173,7 @@ private void CreateStore() }; store = new(kvSettings - , StoreFunctions.Create(new ByteArrayKeyComparer(), () => new Tsavorite.core.ByteArrayBinaryObjectSerializer(), () => new MyGarnetObjectSerializer()) + , StoreFunctions.Create(new SpanByteComparer(), () => new MyGarnetObjectSerializer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); } } @@ -169,10 +181,10 @@ private void CreateStore() /// /// Serializer for IGarnetObject /// - sealed class MyGarnetObjectSerializer : BinaryObjectSerializer + sealed class MyGarnetObjectSerializer : BinaryObjectSerializer { /// - public override void Deserialize(out IGarnetObject obj) + public override void Deserialize(out IHeapObject obj) { var type = (GarnetObjectType)reader.ReadByte(); obj = type switch @@ -186,12 +198,12 @@ public override void Deserialize(out IGarnetObject obj) } /// - public override void Serialize(ref IGarnetObject obj) + public override void Serialize(IHeapObject obj) { if (obj == null) writer.Write((byte)GarnetObjectType.Null); else - obj.Serialize(writer); + ((IGarnetObject)obj).Serialize(writer); } } } \ No newline at end of file diff --git a/test/Garnet.test/IndexGrowthTests.cs b/test/Garnet.test/IndexGrowthTests.cs index 785866f85f9..7b9b9a9fa96 100644 --- a/test/Garnet.test/IndexGrowthTests.cs +++ b/test/Garnet.test/IndexGrowthTests.cs @@ -13,7 +13,7 @@ namespace Garnet.test public class IndexGrowthTests { GarnetServer server; - private int indexResizeTaskDelaySeconds = 10; + private int indexResizeTaskDelaySeconds = 5; private int indexResizeWaitCycles = 2; [SetUp] diff --git a/test/Garnet.test/LuaScriptTests.cs b/test/Garnet.test/LuaScriptTests.cs index e2f5b4ba01a..c165763e602 100644 --- a/test/Garnet.test/LuaScriptTests.cs +++ b/test/Garnet.test/LuaScriptTests.cs @@ -43,7 +43,7 @@ public override bool Execute(TGarnetApi garnetApi, ref CustomProcedu ref var arg = ref procInput.parseState.GetArgSliceByRef(0); var mem = MemoryPool.Rent(arg.Length + 2); - arg.ReadOnlySpan.CopyTo(mem.Memory.Span); + arg. ReadOnlySpan.CopyTo(mem.Memory.Span); mem.Memory.Span[arg.Length] = (byte)'\r'; mem.Memory.Span[arg.Length + 1] = (byte)'\n'; diff --git a/test/Garnet.test/RespCustomCommandTests.cs b/test/Garnet.test/RespCustomCommandTests.cs index e92ba39a677..22f6d075984 100644 --- a/test/Garnet.test/RespCustomCommandTests.cs +++ b/test/Garnet.test/RespCustomCommandTests.cs @@ -40,7 +40,7 @@ static bool ResetBuffer(TGarnetApi garnetApi, ref MemoryResult output, int var buffOffset = garnetApi.GetScratchBufferOffset(); for (var i = 0; i < 120_000; i++) { - garnetApi.GET(key, out var outval); + garnetApi.GET(key, out PinnedSpanByte outval); if (i % 100 == 0) { if (!ResetBuffer(garnetApi, ref output, buffOffset)) @@ -49,8 +49,8 @@ static bool ResetBuffer(TGarnetApi garnetApi, ref MemoryResult output, int } buffOffset = garnetApi.GetScratchBufferOffset(); - garnetApi.GET(key, out var outval1); - garnetApi.GET(key, out var outval2); + garnetApi.GET(key, out PinnedSpanByte outval1); + garnetApi.GET(key, out PinnedSpanByte outval2); if (!ResetBuffer(garnetApi, ref output, buffOffset)) return false; buffOffset = garnetApi.GetScratchBufferOffset(); @@ -79,7 +79,7 @@ public override void Main(TGarnetApi garnetApi, ref CustomProcedureI var buffOffset = garnetApi.GetScratchBufferOffset(); for (int i = 0; i < 120_000; i++) { - garnetApi.GET(key, out var outval); + garnetApi.GET(key, out PinnedSpanByte outval); if (i % 100 == 0) { if (!garnetApi.ResetScratchBuffer(buffOffset)) @@ -100,10 +100,10 @@ public override bool Execute(TGarnetApi garnetApi, ref CustomProcedu var key = GetNextArg(ref procInput, ref offset); var buffOffset1 = garnetApi.GetScratchBufferOffset(); - garnetApi.GET(key, out var outval1); + garnetApi.GET(key, out PinnedSpanByte outval1); var buffOffset2 = garnetApi.GetScratchBufferOffset(); - garnetApi.GET(key, out var outval2); + garnetApi.GET(key, out PinnedSpanByte outval2); if (!garnetApi.ResetScratchBuffer(buffOffset1)) { @@ -153,7 +153,7 @@ public override bool Execute(TGarnetApi garnetApi, ref CustomProcedu garnetApi.Increment(keyToIncrement, out long _, 1); var keyToReturn = GetNextArg(ref procInput, ref offset); - garnetApi.GET(keyToReturn, out ArgSlice outval); + garnetApi.GET(keyToReturn, out PinnedSpanByte outval); WriteBulkString(ref output, outval.Span); return true; } @@ -178,7 +178,7 @@ public override unsafe void Main(TGarnetApi garnetApi, ref CustomPro // key will have an etag associated with it already but the transaction should not be able to see it. // if the transaction needs to see it, then it can send GET with cmd as GETWITHETAG - garnetApi.GET(key, out ArgSlice outval); + garnetApi.GET(key, out PinnedSpanByte outval); List valueToMessWith = outval.ToArray().ToList(); @@ -203,18 +203,16 @@ public override unsafe void Main(TGarnetApi garnetApi, ref CustomPro RawStringInput input = new RawStringInput(RespCommand.SET); input.header.cmd = RespCommand.SET; // if we send a SET we must explictly ask it to retain etag, and use conditional set - input.header.SetWithEtagFlag(); + input.header.SetWithETagFlag(); fixed (byte* valuePtr = valueToMessWith.ToArray()) { - ArgSlice valForKey1 = new ArgSlice(valuePtr, valueToMessWith.Count); + PinnedSpanByte valForKey1 = PinnedSpanByte.FromPinnedPointer(valuePtr, valueToMessWith.Count); input.parseState.InitializeWithArgument(valForKey1); // since we are setting with retain to etag, this change should be reflected in an etag update - SpanByte sameKeyToUse = key.SpanByte; - garnetApi.SET_Conditional(ref sameKeyToUse, ref input); + garnetApi.SET_Conditional(key, ref input); } - var keyToIncrment = GetNextArg(ref procInput, ref offset); // for a non SET command the etag should be invisible and be updated automatically diff --git a/test/Garnet.test/RespEtagTests.cs b/test/Garnet.test/RespEtagTests.cs index abc7b88c959..c09f90e6266 100644 --- a/test/Garnet.test/RespEtagTests.cs +++ b/test/Garnet.test/RespEtagTests.cs @@ -14,7 +14,7 @@ namespace Garnet.test { [TestFixture] - public class RespEtagTests + public class RespETagTests { private GarnetServer server; private Random r; diff --git a/test/Garnet.test/RespSortedSetTests.cs b/test/Garnet.test/RespSortedSetTests.cs index a814ef343a1..794bfb623e9 100644 --- a/test/Garnet.test/RespSortedSetTests.cs +++ b/test/Garnet.test/RespSortedSetTests.cs @@ -19,12 +19,12 @@ namespace Garnet.test { - using TestBasicGarnetApi = GarnetApi, - SpanByteAllocator>>, - BasicContext>, - GenericAllocator>>>>; + using TestBasicGarnetApi = GarnetApi, + SpanByteAllocator>>, + BasicContext, + ObjectAllocator>>>; [TestFixture] public class RespSortedSetTests @@ -104,12 +104,12 @@ public unsafe void SortedSetPopTest() var key = Encoding.ASCII.GetBytes("key1"); fixed (byte* keyPtr = key) { - var result = api.SortedSetPop(new ArgSlice(keyPtr, key.Length), out var items); + var result = api.SortedSetPop(PinnedSpanByte.FromPinnedPointer(keyPtr, key.Length), out var items); ClassicAssert.AreEqual(1, items.Length); ClassicAssert.AreEqual("a", Encoding.ASCII.GetString(items[0].member.ReadOnlySpan)); ClassicAssert.AreEqual("1", Encoding.ASCII.GetString(items[0].score.ReadOnlySpan)); - result = api.SortedSetPop(new ArgSlice(keyPtr, key.Length), out items); + result = api.SortedSetPop(PinnedSpanByte.FromPinnedPointer(keyPtr, key.Length), out items); ClassicAssert.AreEqual(1, items.Length); ClassicAssert.AreEqual("b", Encoding.ASCII.GetString(items[0].member.ReadOnlySpan)); ClassicAssert.AreEqual("2", Encoding.ASCII.GetString(items[0].score.ReadOnlySpan)); @@ -136,7 +136,7 @@ public unsafe void SortedSetPopWithExpire() var key = Encoding.ASCII.GetBytes("key1"); fixed (byte* keyPtr = key) { - var result = api.SortedSetPop(new ArgSlice(keyPtr, key.Length), out var items); + var result = api.SortedSetPop(PinnedSpanByte.FromPinnedPointer(keyPtr, key.Length), out var items); ClassicAssert.AreEqual(1, items.Length); ClassicAssert.AreEqual("b", Encoding.ASCII.GetString(items[0].member.ReadOnlySpan)); ClassicAssert.AreEqual("2", Encoding.ASCII.GetString(items[0].score.ReadOnlySpan)); diff --git a/test/Garnet.test/RespTests.cs b/test/Garnet.test/RespTests.cs index 8594a918284..b9abdf0c5ec 100644 --- a/test/Garnet.test/RespTests.cs +++ b/test/Garnet.test/RespTests.cs @@ -1729,7 +1729,7 @@ public void MultipleExistsKeysAndObjects() #region ExpireTime [Test] - public void ExpiretimeWithStingValue() + public void ExpiretimeWithStringValue() { using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); var db = redis.GetDatabase(0); @@ -3774,7 +3774,7 @@ public void AppendTest() var val = "myKeyValue"; var val2 = "myKeyValue2"; - db.StringSet(key, val); + _ = db.StringSet(key, val); var len = db.StringAppend(key, val2); ClassicAssert.AreEqual(val.Length + val2.Length, len); @@ -3782,7 +3782,7 @@ public void AppendTest() ClassicAssert.AreEqual(val + val2, _val.ToString()); // Test appending an empty string - db.StringSet(key, val); + _ = db.StringSet(key, val); var len1 = db.StringAppend(key, ""); ClassicAssert.AreEqual(val.Length, len1); @@ -3796,23 +3796,50 @@ public void AppendTest() _val = db.StringGet(nonExistentKey); ClassicAssert.AreEqual(val2, _val.ToString()); + } + + [Test] + public void AppendLargeStringValueTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var key1 = "myKey1"; + var key2 = "myKey2"; + var val2 = "myKeyValue2"; - // Test appending to a key with a large value var largeVal = new string('a', 1000000); - db.StringSet(key, largeVal); - var len3 = db.StringAppend(key, val2); - ClassicAssert.AreEqual(largeVal.Length + val2.Length, len3); - // Test appending to a key with metadata - var keyWithMetadata = "keyWithMetadata"; - db.StringSet(keyWithMetadata, val, TimeSpan.FromSeconds(10000)); - var len4 = db.StringAppend(keyWithMetadata, val2); - ClassicAssert.AreEqual(val.Length + val2.Length, len4); + // Test appending to a key with a large value + _ = db.StringSet(key1, largeVal); + var len = db.StringAppend(key1, val2); + ClassicAssert.AreEqual(largeVal.Length + val2.Length, len); + + // Test appending a large value to a key + _ = db.StringSet(key2, val2); + var len2 = db.StringAppend(key2, largeVal); + ClassicAssert.AreEqual(largeVal.Length + val2.Length, len); + } - _val = db.StringGet(keyWithMetadata); + [Test] + public void AppendWithExpirationTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var key = "keyWithExpiration"; + var val = "myKeyValue"; + var val2 = "myKeyValue2"; + + // Test appending to a key with expiration + _ = db.StringSet(key, val, TimeSpan.FromSeconds(10000)); + var len = db.StringAppend(key, val2); + ClassicAssert.AreEqual(val.Length + val2.Length, len); + + var _val = db.StringGet(key); ClassicAssert.AreEqual(val + val2, _val.ToString()); - var time = db.KeyTimeToLive(keyWithMetadata); + var time = db.KeyTimeToLive(key); ClassicAssert.IsTrue(time.Value.TotalSeconds > 0); } diff --git a/test/Garnet.test/TestProcedureBitmap.cs b/test/Garnet.test/TestProcedureBitmap.cs index 9e12a7cd354..e8f77cdca03 100644 --- a/test/Garnet.test/TestProcedureBitmap.cs +++ b/test/Garnet.test/TestProcedureBitmap.cs @@ -84,7 +84,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p result = false; goto returnTo; } - api.GET(destinationKeyBitOp, out var valueData); + api.GET(destinationKeyBitOp, out PinnedSpanByte valueData); var actualResultBitOp = BitConverter.ToInt64(valueData.ToArray(), 0); long expectedResultBitOp = ~src; diff --git a/test/Garnet.test/TestProcedureHash.cs b/test/Garnet.test/TestProcedureHash.cs index cc51de27002..7c4ffa775c2 100644 --- a/test/Garnet.test/TestProcedureHash.cs +++ b/test/Garnet.test/TestProcedureHash.cs @@ -40,8 +40,8 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput procInput) where TGarnetApi : IGarnetApi { var offset = 0; - var pairs = new (ArgSlice field, ArgSlice value)[6]; - var fields = new ArgSlice[pairs.Length]; + var pairs = new (PinnedSpanByte field, PinnedSpanByte value)[6]; + var fields = new PinnedSpanByte[pairs.Length]; var myHash = GetNextArg(ref procInput.parseState, ref offset); diff --git a/test/Garnet.test/TestProcedureLists.cs b/test/Garnet.test/TestProcedureLists.cs index 37e5c10e8db..755c02375bd 100644 --- a/test/Garnet.test/TestProcedureLists.cs +++ b/test/Garnet.test/TestProcedureLists.cs @@ -43,7 +43,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput procInput) where TGarnetApi : IGarnetApi { var offset = 0; - var elements = new ArgSlice[10]; + var elements = new PinnedSpanByte[10]; var lstKeyA = GetNextArg(ref procInput, ref offset); var lstKeyB = GetNextArg(ref procInput, ref offset); diff --git a/test/Garnet.test/TestProcedureSet.cs b/test/Garnet.test/TestProcedureSet.cs index 2f33531bd42..06d42c57e7a 100644 --- a/test/Garnet.test/TestProcedureSet.cs +++ b/test/Garnet.test/TestProcedureSet.cs @@ -39,7 +39,7 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput procInput) where TGarnetApi : IGarnetApi { var offset = 0; - var elements = new ArgSlice[10]; + var elements = new PinnedSpanByte[10]; var setA = GetNextArg(ref procInput, ref offset); diff --git a/test/Garnet.test/TestProcedureSortedSets.cs b/test/Garnet.test/TestProcedureSortedSets.cs index 14c949a4543..a3e35334c9f 100644 --- a/test/Garnet.test/TestProcedureSortedSets.cs +++ b/test/Garnet.test/TestProcedureSortedSets.cs @@ -41,8 +41,8 @@ public override void Main(TGarnetApi api, ref CustomProcedureInput p private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput procInput) where TGarnetApi : IGarnetApi { var offset = 0; - var ssItems = new (ArgSlice score, ArgSlice member)[10]; - var ssMembers = new ArgSlice[10]; + var ssItems = new (PinnedSpanByte score, PinnedSpanByte member)[10]; + var ssMembers = new PinnedSpanByte[10]; var ssA = GetNextArg(ref procInput, ref offset); @@ -57,7 +57,7 @@ private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput var maxRange = GetNextArg(ref procInput, ref offset); var match = GetNextArg(ref procInput, ref offset); - var ssB = new ArgSlice(); + var ssB = new PinnedSpanByte(); api.SortedSetAdd(ssB, ssItems[0].score, ssItems[0].member, out int count); if (count != 0) return false; @@ -73,7 +73,7 @@ private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput var strMatch = Encoding.ASCII.GetString(match.ReadOnlySpan); // Exercise SortedSetScan - api.SortedSetScan(ssA, 0, strMatch, ssItems.Length, out ArgSlice[] itemsInScan); + api.SortedSetScan(ssA, 0, strMatch, ssItems.Length, out PinnedSpanByte[] itemsInScan); // The pattern "*em*" should match all items if (itemsInScan.Length != (ssItems.Length * 2) + 1) @@ -114,15 +114,15 @@ private static bool TestAPI(TGarnetApi api, ref CustomProcedureInput if (status != GarnetStatus.OK || newScore != 12345) return false; - status = api.SortedSetExpire(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), ArgSlice.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], DateTimeOffset.UtcNow.AddMinutes(10), ExpireOption.None, out var expireResults); + status = api.SortedSetExpire(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), PinnedSpanByte.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], DateTimeOffset.UtcNow.AddMinutes(10), ExpireOption.None, out var expireResults); if (status != GarnetStatus.OK || expireResults.Length != 2 || expireResults[0] != 1 || expireResults[1] != -2) return false; - status = api.SortedSetTimeToLive(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), ArgSlice.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], out var expireIn); + status = api.SortedSetTimeToLive(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), PinnedSpanByte.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], out var expireIn); if (status != GarnetStatus.OK || expireIn.Length != 2 || expireIn[0].TotalMilliseconds <= 0 || expireIn[0].TotalMilliseconds > TimeSpan.FromMinutes(10).TotalMilliseconds || expireIn[1].TotalMilliseconds != 0) return false; - status = api.SortedSetPersist(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), ArgSlice.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], out var persistResults); + status = api.SortedSetPersist(ssA, [.. ssItems.Skip(4).Take(1).Select(x => x.member), PinnedSpanByte.FromPinnedSpan(Encoding.UTF8.GetBytes("nonExist"))], out var persistResults); if (status != GarnetStatus.OK || persistResults.Length != 2 || persistResults[0] != 1 || persistResults[1] != -2) return false; diff --git a/website/docs/dev/tsavorite/epochprotection.md b/website/docs/dev/tsavorite/epochprotection.md index 8db3babe0a5..e21bbe86481 100644 --- a/website/docs/dev/tsavorite/epochprotection.md +++ b/website/docs/dev/tsavorite/epochprotection.md @@ -8,7 +8,7 @@ title: Epoch Protection Framework ## Context -We need to ensure shared variables are not being read and mutated simultaneously without determinisitic orderings. Commonly used concurreny primitives such as Mutexes and sempahores provided by the language require threads to synchronize frequently with each other. This Synchronization across threads is expensive; Epoch protection **reduces the frequency of synchronization across threads**. +We need to ensure shared variables are not being read and mutated simultaneously without deterministic orderings. Commonly used concurreny primitives such as Mutexes and semaphores provided by the language require threads to synchronize frequently with each other. This Synchronization across threads is expensive; Epoch protection **reduces the frequency of synchronization across threads**. ## Epoch Protection (10,000-foot view) diff --git a/website/docs/dev/tsavorite/locking.md b/website/docs/dev/tsavorite/locking.md index fc7af97e04d..7b6e0e501de 100644 --- a/website/docs/dev/tsavorite/locking.md +++ b/website/docs/dev/tsavorite/locking.md @@ -6,19 +6,17 @@ title: Locking # Locking -There are three modes of locking in Tsavorite, set by a `ConcurrencyControlMode` value on the Tsavorite constructor: - - `LockTable`: Tsavorite's hash index buckets are used to hold the lock state. Locktable locking is either manual or transient: - - **Manual**: Garnet calls a `Lock` method on `LockableContext` or `LockableUnsafeContext` (hereafter referred to collectively as `Lockable*Context`) at the beginning of a transaction, passing an ordered array of keys, and must call `Unlock` when the transaction is complete. Tsavorite does not try to lock during individual operations on these session contexts. - - **Transient**: Tsavorite acquires and releases locks for individual keys for the duration of a data operation: Upsert, RMW, Read, or Delete. Collectively, these are referred to here as `InternalXxx` for the internal methods that implement them. - - `None`: No locking is done by Tsavorite. +Locking is always on in Tsavorite. It is done by locking the HashIndex bucket. There are two modes of locking; these are automatic based on what sessions the caller uses: + - **Manual**: In this mode, the Garnet processing layer calls a `Lock` method on `TransactionalContext` or `TransactionalUnsafeContext` (hereafter referred to collectively as `Transactional*Context`) at the beginning of a transaction, passing an ordered array of keys, and must call `Unlock` when the transaction is complete. Tsavorite does not try to lock during individual operations on these session contexts. + - **Transient**: Tsavorite acquires and releases locks for individual keys for the duration of a data operation: Upsert, RMW, Read, or Delete. Collectively, these are referred to here as `InternalRUMD` for the internal methods that implement them: Read, Upsert, rMw, and Delete. All locks are obtained via spinning on `Interlocked.CompareExchange` and `Thread.Yield()` and have limited spin count, to avoid deadlocks; if they fail to acquire the desired lock in this time, the operation retries. -As noted above, manual locking is done by obtaining the `Lockable*Context` instance from a `ClientSession`. There are currently 4 `*Context` implementations; all are `struct` for inlining. All `*Context` are obtained as properties on the `ClientSession` named for the type (e.g. `clientSession.LockableContext`). The characteristics of each `*Context` are: +As noted above, manual locking is done by obtaining the `Transactional*Context` instance from a `ClientSession`. There are currently 4 `*Context` implementations; all are `struct` for inlining. All `*Context` are obtained as properties on the `ClientSession` named for the type (e.g. `clientSession.TransactionalContext`). The characteristics of each `*Context` are: - **`BasicContext`**: This is exactly the same as `ClientSession`, internally calling directly through to `ClientSession`'s methods and reusing `ClientSession`'s `TsavoriteSession`. It provides safe epoch management (acquiring and releasing the epoch on each call) and Transient locking. - **`UnsafeContext : IUnsafeContext`**: This provides Transient locking, but rather than safe epoch management handled per-operation by Tsavorite, this supports "unsafe" manual epoch management controlled by the client via `BeginUnsafe()` and `EndUnsafe()`; it is the client's responsibility to make these calls correctly. `UnsafeContext` API methods call the internal ContextRead etc. methods without doing the Resume and Suspend (within try/finally) of epoch protection as is done by the "Safe" API methods. -- **`LockableContext : ILockableContext`**: This provides safe epoch management, but rather than Transient locking, this requires Manual locks via `BeginLockable` and `EndLockable`. This requirement ensures that all locks are acquired before any methods accessing those keys are called. -- **`LockableUnsafeContext : ILockableContext, IUnsafeContext`**: This combines manual epoch management and manual locking, exposing both sets of methods. +- **`TransactionalContext : ITransactionalContext`**: This provides safe epoch management, but rather than Transient locking, this requires Manual locks via `BeginTransactional` and `EndTransactional`. This requirement ensures that all locks are acquired before any methods accessing those keys are called. +- **`TransactionalUnsafeContext : ITransactionalContext, IUnsafeContext`**: This combines manual epoch management and manual locking, exposing both sets of methods. In addition to the `Lock` methods, Tsavorite supports: - `TryLock`: Accepts an array of keys and returns true if all locks were acquired, else false (and any locks that were acquired are released) @@ -29,28 +27,28 @@ In addition to the `Lock` methods, Tsavorite supports: All manual locking of keys must lock the keys in a deterministic order, and unlock in the reverse order, to avoid deadlocks. Lock spinning is limited in order to avoid deadlocks such as the following: - - `Lockable*Context` LC1 exclusively locks k1 + - `Transactional*Context` LC1 exclusively locks k1 - `BasicContext` BC1 tries to acquire an exclusive Transient lock on k1, and spins while holding the epoch - LC1 does an RMW on k1 resulting in a CopyUpdate; this does a BlockAllocate that finds it must flush pages from the head of the log in order to make room at the tail. - LC1 therefore calls BumpCurrentEpoch(... OnPagesClosed) - Because BC1 holds the epoch, the OnPagesClosed() call is never drained, so we have deadlock By ensuring that locks are limited in spins, we force one or both of the above sessions to release any locks it has already aquired and return up the callstack to retry the operation via RETRY_LATER (which refreshes the epoch, allowing other operations such as the OnPagesClosed() mentioned above to complete). -Transient locks are never held across pending I/O or other Wait operations. All the data operations' low-level implementors (`InternalRead`, `InternalUpsert`, `InternalRMW`, and `InternalDelete`--collectively known as `InternalXxx`) release these locks when the call is exited; if the operations must be retried, the locks are reacquired as part of the normal operation there. +Transient locks are never held across pending I/O or other Wait operations. All the data operations' low-level implementors (`InternalRead`, `InternalUpsert`, `InternalRMW`, and `InternalDelete`--collectively known as `InternalRUMD`) release these locks when the call is exited; if the operations must be retried, the locks are reacquired as part of the normal operation there. ## Example -Here is an example of the above two use cases, condensed from the unit tests in `LockableUnsafeContextTests.cs`: +Here is an example of the above two use cases, condensed from the unit tests in `TransactionalUnsafeContextTests.cs`: ```cs - var luContext = session.GetLockableUnsafeContext(); + var luContext = session.GetTransactionalUnsafeContext(); luContext.BeginUnsafe(); - luContext.BeginLockable(); + luContext.BeginTransaction(); var keys = new[] { - new FixedLengthLockableKeyStruct(readKey24, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(readKey51, LockType.Shared, luContext), // Source, shared - new FixedLengthLockableKeyStruct(resultKey, LockType.Exclusive, luContext), // Destination, exclusive + new FixedLengthTransactionalKeyStruct(readKey24, LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(readKey51, LockType.Shared, luContext), // Source, shared + new FixedLengthTransactionalKeyStruct(resultKey, LockType.Exclusive, luContext), // Destination, exclusive }; // Sort the keys to guard against deadlock @@ -64,7 +62,7 @@ Here is an example of the above two use cases, condensed from the unit tests in luContext.Unlock(keys); - luContext.EndLockable(); + luContext.EndTransaction(); luContext.EndUnsafe(); ``` @@ -73,7 +71,7 @@ Here is an example of the above two use cases, condensed from the unit tests in This section covers the internal design and implementation of Tsavorite's locking. ### Operation Data Structures -There are a number of variables necessary to track the main hash table entry information, the 'source' record as defined above, and other stack-based data relevant to the operation. These variables are placed within structs that live on the stack at the `InternalXxx` level. +There are a number of variables necessary to track the main hash table entry information, the 'source' record as defined above, and other stack-based data relevant to the operation. These variables are placed within structs that live on the stack at the `InternalRUMD` level. #### HashEntryInfo This is used for hash-chain traversal and CAS updates. It consists primarily of: @@ -134,9 +132,7 @@ Some relevant `RecordInfo` bits: ### Locking Flow -When `Internalxxx` when `ConcurrencyControlMode` is `LockTable`: - -We obtain the key hash at the start of the operation, so we lock its bucket if we are not in a `Lockable*Context` (if we are, we later Assert that the key is already locked). +We obtain the key hash at the start of the operation, so we lock its bucket if we are not in a `Transactional*Context` (if we are, we later Assert that the key is already locked). Following this, the requested operation is performed within a try/finally block whose 'finally' releases the lock. @@ -161,4 +157,4 @@ Using the above example and assuming an update of r8000, the resulting chain wou - `HashTable` -> r8000 (invalid) -> r7000 -> mxxxx (new) -> m4000 -> m3000 -> m... - In this example, note that the record address spaces are totally different between the main log and readcache; "xxxx" is used as the "new address" to symbolize this. -This splicing operation requires that we deal with updates at the tail of the tag chain (in the `HashEntryInfo`) as well as at the splice point. This cannot be done as a single atomic operation. To handle this, we detach the readcache prefix chain, insert the new record at the tail, and then reattach the detached records. See `DetachAndReattachReadCacheChain` for specifics. We may fail the reattach, but this is acceptable (versus more complicated and expensive locking) because such failures should be rare and the readcache is just a performance optimization. +This splicing operation requires that we deal with updates at the tail of the tag chain (in the `HashEntryInfo`) as well as at the splice point. This cannot be done as a single atomic operation. To handle this, we check if another session added a readcache entry from a pending read while we were inserting a record at the tail of the log. If so, the new readcache record must be invalidated (see `ReadCacheCheckTailAfterSplice`). diff --git a/website/docs/dev/tsavorite/logrecord.md b/website/docs/dev/tsavorite/logrecord.md new file mode 100644 index 00000000000..08776155f1f --- /dev/null +++ b/website/docs/dev/tsavorite/logrecord.md @@ -0,0 +1,164 @@ +--- +id: logrecord +sidebar_label: LogRecord +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 LogRecords 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. + +## `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. + +There are still areas where direct `byte*` are used, such as number parsing. Later work can revisit this to use `(ReadOnly)Span` instead if there is no performance impact. + +The `SpanByte` class now exists only as a static utility class that provides extension functions `(ReadOnly)Span`. + +## All Keys are now `ReadOnlySpan` at the Tsavorite Level + +Originally, Tsavorite was templated on the `TKey` generic type, which was either `SpanByte` for the string store or `byte[]` for the object store. In this revision, all keys at the Tsavorite level are now `ReadOnlySpan`. At the Garnet processing level, they may be `PinnedSpanByte` at the `GarnetApi` layer and above. Any key structure must be converted to a stream of bytes and a `ReadOnlySpan` or `PinnedSpanByte` created over this. This can be a stack variable (which is not subject to GC) or a pinned pointer. + +This has simplified the signature and internal implementation of TsavoriteKV itself, the sessions, allocators, ISessionFunctions, Compaction, Iterators, and so on. And we now have only two allocators, `SpanByteAllocator` and the new `ObjectAllocator`. + +## Removal of `BlittableAllocator` + +As part of the migration to `SpanByte`-only keys, `BlittableAllocator` has been removed. Tsavorite Unit Tests such as `BasicTests` and the YCSB benchmark's fixed-length test illustrate simple ways to use stack-based 'long' keys and values with `SpanByte`. This does incur some log record space overhead for the key's or value's length prefix. + +## Replace `GenericAllocator` with `ObjectAllocator` + +With the move to `SpanByte`-only keys we also created a new `ObjectAllocator` for a store that uses an object value type. `GenericAllocator` is not able to take SpanByte keys, and it also uses a separate object log file. `ObjectAllocator` uses a single log file; see the separate `ObjectAllocator` documentation for more details. + +### IHeapObject + +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: +- 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. + +Garnet ensures that each operation on an object (such as a `SortedSet`) keeps these fields up to date. + +### `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`. + +The `ObjectIdMap` manages the lifetime of .NET objects for the `ObjectAllocator`. Because we cannot store objects in the unmanaged log (or IntPtrs, as they will become obsolete), we store a 4-byte `ObjectId` in the log record; this is an index into the `ObjectIdMap`. These IDs are simply integer indices into the `MultiLevelPageArray` `objectArray` of the `ObjectIdMap`, so it is not truly a "map". + +The `ObjectIdMap` has a freeList in a `SimpleConcurrentStack` called `freeSlots`. When an object is deallocated, its slot is nulled and the slot is added to the free list. When an object is allocated, it looks for a free slot in the free list, and if it finds one, it returns that slot. If it does not find one, it allocates a new slot and returns that. This freelist keeps the `ObjectIdMap` from having to grow more than necessary. + +#### MultiLevelPageArray + +The `MultiLevelPageArray` is a data structure to provide efficiently growable arrays with direct indexing. It is a managed structure, as it is used by the `ObjectIdMap` to root .NET objects. It is also used to provide a structure for simple stacks such as for free lists. + +This `MultiLevelPageArray` is a 3-d array of page vectors. Because `NativePageAllocator` allocates pages for caller use, this can be envisioned as a book, where the first two dimensions are infrastructure, and the third is where the user-visible allocations are created. + - The first dimension is the "book", which is a collection of "chapters". Think of the book as a spine, which can be reallocated--but when it is reallocated, the individual chapters, and references within them, are not moved, so may be accessed by other threads. + - The second dimension is the "chapters", which are collections of pages. + - The third dimension is the actual pages of data which are returned to the user. "Page" is somewhat of a misnomer, as the purpose has changed slightly from its initial intent; currently these are object slots for `IHeapObject` and `byte[]`, as well as integer indexes for freelists. + +This structure is chosen so that only the "book" is grown; individual chapters are allocated as a fixed size. This means that getting and clearing items in the chapter does not have to take a latch to prevent a lost update as the array is grown, as would be necessary if there was only a single level of infrastructure (i.e. a growable single vector). + +The `MultiLevelPageArray` is a managed structure, because it is also used to hold the .NET objects for the `ObjectAllocator` to manage their lifetimes. + +In the initial implementation `MultiLevelPageArray` has fixed-size book (1024) and chapters (64k), but this can be made configurable. + +#### SimpleConcurrentStack + +The `SimpleConcurrentStack` sits on top of the `MultiLevelPageArray` and provides a simple stack interface for the free lists. + +## Overflow Keys and Values + +To keep the size of the main log record tractable, we provide an option for `ObjectAllocator` to have large `Span` keys and values "overflow"; rather than being stored inline in the log record, they are allocated separately as `byte[]`, and an integer `ObjectId` is stored in the log record (with the key or value having no explicit length, and the data being the `ObjectId` of size `sizeof(int)`). This redirection does incur some performance overhead. The initial reason for this was to keep `ObjectAllocator` pages small enough that the page-level object size tracking would be sufficiently granular; if those pages are large enough to support large keys, then it is possible there are a large number of records with small keys and large objects, making it impossible to control object space budgets with sufficient granularity. By providing this for `Span` values as well, it allows similar control of the number of records per page. + +## ISourceLogRecord + +In this revision of Tsavorite, the individual "ref key" and "ref value" (as well as "ref recordInfo") parameters to `ISessionFunctions` methods have been replaced by a single `LogRecord` parameter. Not only does this consolidate those record attributes, it also encapsulate the "optional" record attributes of `ETag` and `Expiration`, as well as managing the `FillerLength` that allows records to shrink and re-expand in place. Previously the `ISessionFunctions` implementation had to manage the "extra" length; that is now automatically handled by the `LogRecord`. Similarly, `ETag` and `Expiration` previously were encoded into the Value `SpanByte` or a field of the object and this required tracking additional metadata and shifting when these values were added/removed; these too are now managed by the `LogRecord` as first-class properties. + +As part of this change, keys are now always `ReadOnlySpan` at the Tsavorite level. At the processing layer, they are initially `PinnedSpanBytes`; these have a `ReadOnlySpan` property that is called to convert them to `ReadOnlySpan` at the GarnetApi/StorageApi boundary. + +Although we have two allocators, there is only one `LogRecord` family; we do not have separate `StringLogRecord` and `ObjectLogRecord`. There are a couple reasons for this: + - It would be more complex to maintain them, especially as we have multiple implementations of `ISourceLogRecord`. + - Iterators would no longer be able to iterate both stores. + - The `ObjectAllocator` can have `SpanByte`, overflow `byte[]`, or `IHeapObject` values, so the `LogRecord` must be able to handle both. +This decision may be revisited in the future; for example, `SpanByteAllocator` currently cannot have overflow keys or values, so a much leaner implementation could be used for that case. This would require a `TLogRecord` generic type in place of the `TKey` and `TValue` types that have been removed in this revision. + +`ISourceLogRecord` defines the common operations among a number of `LogRecord` implementations. These common operations are summarized here, and the implementations are described below. + - Obtaining the RecordInfo header. There is both a "ref" (mutable) and non-"ref" (readonly) form. + - Obtaining the `ReadOnlySpan` of the Key. + - Obtaining the ValueSpan `Span` (for both `SpanByteAllocator` and `ObjectAllocator`; this may be either inline or overflow). + - Obtaining the ValueObject (for `ObjectAllocator` only. This is intended to be an `IHeapObject`. + - Obtaining the "optionals": ETag and Expiration. Note that while `FillerLength` is also optional in the record (it may or may not be present), it is now completely handled by the `LogRecord` and thus is unknown to the caller. + - Setting a new Value Span or Object. + - Setting the Value field's length. + - This is done automatically when setting the Value Span or Object. + - It may also be called directly and then the ValueSpan obtained and operated on directly, rather than creating a separate `Span` and copying. + - Utilities such as clearing the Value Object, obtaining record size info, and so on. + +For operations that take an input log record, such as `ISessionFunctions.CopyUpdater`, the input log record is of type `TSourceLogRecord`, which may be either `LogRecord` or `DiskLogRecord`. No code outside Tsavorite should need to know the actual type. Within Tsavorite, it is sometimes useful to call `AsLogRecord` or `AsDiskLogRecord` and operate accordingly. + +### LogRecord struct + +This is the primary implementation which wraps a log record. It carries the log record's physical address and, if this is an `ObjectAllocator` record, an `ObjectIdMap` for that log record's log page. See `LogRecord.cs` for more details, including the record layout and comments. + +In the `ObjectAllocator`, `TrySetValueLength` also manages conversion between the three Value "styles". Both Keys and Values may be inline or overflow, and values additionally may be object. Keys are not mutable, so there is no `LogRecord` method to change them. Values, however, may move between any of the three: + - Initially, a Value in the `ObjectAllocator` may be a small inline value, such as the first couple strings of a list. This is stored as a length-prefixed byte stream "inline" in the record. + - Depending on the inline size limit, such a value may overflow, and become a pointer to an `OverflowAllocator` allocation. In this case, `TrySetValueLength` will handle converting the inline field value to an overflow pointer, shrinking the record, moving the optionals, and adjusting the `FillerLength` as needed. The record has no length prefix for this; its inline size cost is simply an 8-byte pointer (`SpanField.OverflowInlineSize` is the constant used). + - Finally, the value may be "promoted" to an actual object; e.g., allocating a `ListObject` and populating it from the `ValueSpan`. Again, `TrySetValueLength` will handle this conversion, resizing the Value, moving the optionals, and adjusting the `FillerLength` as needed. The record has no length prefix for this; its inline size cost is simply a 4-byte int containing the `ObjectId` for the `ObjectIdMap`(`ObjectIdMap.ObjectIdSize` is the constant used). + + `TrySetValueLength` handles this switching between inline, overflow, and object values automatically, based upon settings in the `RecordSizeInfo` that is also passed to `ISessionFunctions` methods and then to the `LogRecord`. When `LogRecord` converts between these styles, it handles all the necessary freeing and allocations. For example, when growing a Value causes allows it to move from inline to overflow, the `byte[]` slot is allocated in `ObjectIdMap`; if it shrinks enough to return to inline, the `ObjectIdMap` slot element is nulled and the slot is added to the freelist, and the record is resized to inline. + + Although `TrySetValueLength` allocates the `ObjectId` slot, it does not know the actual object, so the `ISessionFunctions` implementation must create the object and call `TrySetValueObject`. + +#### 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: +- The `RecordSizeInfo` is populated prior to record allocation: + - This is done by calling the appropriate `IVariableLengthInput` method to populate the `RecordFieldInfo` part of the structure with Key and Value sizes, whether the Value is to be an object, and whether there are optionals present: + - `GetRMWModifiedFieldInfo`: For in-place or copy updates via RMW + - `GetRMWInitialFieldInfo`: For initial update via RMW + - `GetUpsertFieldInfo`: For writing via Upsert. There are three overloads of this, because Upsert takes a Value which may be one of `ReadOnlySpan`, `IHeapObject`, or a source `TSourceLogRecord`. + - The allocator's `PopulateRecordSizeInfo` method is called to fill in the `RecordSizeInfo` fields based upon the `RecordFieldInfo` fields and other information such as maximum inline size and so on: + - Whether the Key or Value are inline or overflow + - Utility methods to make it easy for the Tsavorite allocators to calculate the allocation size + - Other utility methods to allow `LogRecord.TrySetValueLength` to operate efficiently. + +#### LogField + +This is a static class that provides utility functions for `LogRecord` to operate on a Key or Value field at a certain address. + +As a terminology note, `LogField` (and `RecordSizeInfo` and `LogRecord`) use the following terms for field layout: +- Inline: The field is stored inline in the record, with a length prefix followed by the byte stream of the actual data. The "Inline size" is the TotalSize; the size of the length prefix plus the length of the data. The "Data size" is the length of the byte stream. +- Overflow: The field is a byte stream that exceeds the limit to remain inline, so is stored in an overflow `byte[]`. The "Inline size" is `sizeof(ObjectId)`, which is an int; there is no field length prefix. The "Data size" is the length of the byte stream. +- Object: The field is an object. As with overflow, the "Inline size" is `sizeof(ObjectId)`, which is an int; there is no field length prefix. The "Data size" is only relevant during serialization; it is the `IHeapObject.DataSize` field. + +### 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. + +### 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`. + +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`. + +### RecordScanIterator + +`RecordScanIterator` must copy in-memory source records for Pull iterations, so it implements `ISourceLogRecord` by delegating to a `DiskLogRecord` that is instantiated over its copy buffer. + +### TsavoriteKVIterator + +`TsavoriteKVIterator` must copy in-memory source records for Pull iterations, so it implements `ISourceLogRecord` by delegating its internal `ITsavoriteScanIterator`s. + +## 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 diff --git a/website/docs/dev/tsavorite/object-allocator.md b/website/docs/dev/tsavorite/object-allocator.md new file mode 100644 index 00000000000..6015036641c --- /dev/null +++ b/website/docs/dev/tsavorite/object-allocator.md @@ -0,0 +1,24 @@ +--- +id: object-allocator +sidebar_label: ObjectAllocator +title: ObjectAllocator +--- + +# ObjectAllocator + +The `ObjectAllocator` replaces the `GenericAllocator` to provide two important improvements: +- It supports `SpanByte` keys. Tsavorite now uses only `SpanByte` keys, and the `GenericAllocator` does not support variable-length keys. +- It replaces the "two log file" approach of `GenericAllocator` with an "expand inline when flushing" approach: + - If the key or value is inline, it remains so + - 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: +- Strings are stored in a version of `TsavoriteKV` that uses a `SpanByteAllocator`. +- Objects are stored in a version of `TsavoriteKV` that uses an `ObjectAllocator`. + +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 + +To be filled in when the implementation is underway. diff --git a/website/docs/dev/tsavorite/reviv.md b/website/docs/dev/tsavorite/reviv.md index b8ad03e7ae7..39b4f679e36 100644 --- a/website/docs/dev/tsavorite/reviv.md +++ b/website/docs/dev/tsavorite/reviv.md @@ -129,16 +129,6 @@ FreeList revivification functions as follows: - Clearing the extra value length and filler and calls `DisposeForRevivification` as described in [Maintaining Extra Value Length](#maintaining-extra-value-length). - Unsealing the record; epoch management guarantees nobody is still executing who saw this record before it went into the free record pool. -### Concurrency Control Considerations for FreeList Revivification -FreeList Revivification requires `ConcurrencyControlMode` not be `ConcurrencyControlMode.None`; otherwise the tag chain can be unstable, with the first record in the tag chain potentially removed and reused by revivification while a thread is tracing back from it: - - Thread1: Get the first record address from the `HashBucketEntry` - - Thread2: Delete and elide the first record, putting it on the revivification FreeList - - Thread3: Revivify that record with a different key, setting its .PreviousAddress - - Thread1: Follows the *former* .PreviousAddress and is now on an entirely different tag chain. -This is *only* possible for the first record in the tag chain (the tail-most record, in the `HashBucketEntry`); we do not elide records in the middle of the tag chain. - -For `ConcurrencyControlMode.LockTable`, because the only current LockTable implementation is via the `HashBucket`s and locking at the `HashBucket` level is higher than the tag chain, we get a stable tag chain *almost* for free. The cost is that the `HashBucket` must be locked *before* calling TracebackForKeyMatch. - ### `FreeRecordPool` Design The FreeList hierarchy consists of: - The `FreeRecordPool`, which maintains the bins, deciding which bin should be used for Enqueue and Dequeue. diff --git a/website/docs/dev/tsavorite/storefunctions.md b/website/docs/dev/tsavorite/storefunctions.md index d22f9b9d1c3..88eefea61ad 100644 --- a/website/docs/dev/tsavorite/storefunctions.md +++ b/website/docs/dev/tsavorite/storefunctions.md @@ -8,7 +8,7 @@ title: StoreFunctions and Allocator Wrapper This section discusses both of these because they were part of a change to add two additional type args, `TStoreFunctions` and `TAllocator`, to `TsavoriteKV` as well as the various sessions and `*Context` (e.g. `BasicContext`). The purpose of both of these is to provide better performance by inlining calls. StoreFunctions also provides better logical design for the location of the operations that are store-level rather than session-level, as described below. -From the caller point of view, we have two new type parameters on `TsavoriteKV`. The `TStoreFunctions` and `TAllocator` are also on `*.Context` (e.g. `BasicContext`) as well. C# allows the 'using' alias only as the first lines of a namespace declaration, and the alias is file-local and recognized by subsequent 'using' aliases, so the "Api" aliases such as `BasicGarnetApi` in multiple files are much longer now. +From the caller point of view, we have two new type parameters on `TsavoriteKV`. The `TStoreFunctions` and `TAllocator` are also on `*.Context` (e.g. `BasicContext`) as well. C# allows the 'using' alias only as the first lines of a namespace declaration, and the alias is file-local and recognized by subsequent 'using' aliases, so the "Api" aliases such as `BasicGarnetApi` in multiple files are much longer now. `TsavoriteKV` constructor has been changed to take 3 parameters: - `KVSettings`. This replaces the previous long list of parameters. `LogSettings`, `ReadCacheSettings`, and `CheckpointSettings` have become internal classes, used only by `TsavoriteKV` (created from `TsavoriteKVSettings`) when instantiating the Allocators (e.g. the new `AllocatorSettings` has a `LogSettings` member). `SerializerSettings` has been removed in favor of methods on `IStoreFunctions`. @@ -22,7 +22,7 @@ These are described in more detail below. Because `IStoreFunctions` is intended to provide maximum inlining, Tsavorite does not provide a `StoreFunctionsBase`. Instead, Tsavorite provides a `StoreFunctions` struct implementation, with optional implementations passed in, for: - Key Comparison (previously passed as an `ITsavoriteKeyComparer` interface, which is not inlined) -- Key and Value Serializers. Due to limitations on type arguments, these must be passed as `Func<>` which creates the implementation instance, and because serialization is an expensive operation, we stay with the `IObjectSerializer` and `IObjectSerializer` interfaces rather than clutter the `IStoreFunctions` interface with the Key and Value Serializer type args. +- Serializers for Value objects. Due to limitations on type arguments, these must be passed as `Func<>` which creates the implementation instance, and because serialization is an expensive operation, we stay with the `IObjectSerializer` interfaces rather than clutter the `IStoreFunctions` interface with the Key and Value Serializer type args. - Record disposal (previously on `ISessionFunctions` as multiple methods, and now only a single method with a "reason" parameter). - Checkpoint completion callback (previously on `ISessionFunctions`). @@ -33,8 +33,8 @@ Of course, because `TsavoriteKV` has the `TStoreFunctions` type parameter, this As with `StoreFunctions`, the Allocator Wrapper is intended to provide maximal inlining. As noted above, type parameters implemented by classes do not generate inlined code; the JITted code is general, for a single `IntPtr`-sized reference. For structs, however, the JITter generates code specific to that specific struct type, in part because the size can vary (e.g. when pushed on the stack as a parameter). There is a hack that allows a type parameter implemented by a class to be inlined: the generic type must be for a struct that wraps the class type and makes calls on that class type in a non-generic way. This is the approach the Allocator Wrapper takes: -- The `BlittableAllocator`, `GenericAllocator`, and `SpanByteAllocator` classes are now the wrapper structs, with `Key`, `Value`, and `TStoreFunctions` type args. These implement the `IAllocator` interface. -- There are new `BlittableAllocatorImpl`, `GenericAllocatorImpl`, and `SpanByteAllocatorImpl` classes that implement most of the functionality as previously, including inheriting from `AllocatorBase`. These also have `Key`, `Value`, and `TStoreFunctions` type args; the `TAllocator` is not needed as a type arg because it is known to be the `XxxAllocator` Wrapper struct. The wrapper structs contain an instance of the `XxxAllocatorImpl` class. +- The `SpanByteAllocator` and `ObjectAllocator` classes are now the wrapper structs, with a `TStoreFunctions` type arg. These implement the `IAllocator` interface. +- There are new `SpanByteAllocatorImpl` and `ObjectAllocatorImpl` classes that implement most of the functionality as previously, including inheriting from `AllocatorBase`. These also have a `TStoreFunctions` type arg; the `TAllocator` is not needed as a type arg because it is known to be the `XxxAllocator` Wrapper struct. The wrapper structs contain an instance of the `XxxAllocatorImpl` class. - `AllocatorBase` itself now contains a `_wrapper` field that is a struct-wrapper instance (which contains the instance pointer of the fully-derived allocator class) that is constrained to the `IAllocator` interface. `AllocatorBase` itself is templated on `TStoreFunctions` and `TAllocator`. The new Allocator definition supports two interfaces: diff --git a/website/sidebars.js b/website/sidebars.js index c54b8db5abc..d4360c76af0 100644 --- a/website/sidebars.js +++ b/website/sidebars.js @@ -24,7 +24,7 @@ const sidebars = { {type: 'category', label: 'Server Extensions', items: ["extensions/overview", "extensions/raw-strings", "extensions/objects", "extensions/transactions", "extensions/procedure", "extensions/module"]}, {type: 'category', label: 'Cluster Mode', items: ["cluster/overview", "cluster/replication", "cluster/key-migration"]}, {type: 'category', label: 'Developer Guide', items: ["dev/onboarding", "dev/code-structure", "dev/configuration", "dev/network", "dev/processing", "dev/garnet-api", - {type: 'category', label: 'Tsavorite - Storage Layer', collapsed: true, items: ["dev/tsavorite/intro", "dev/tsavorite/reviv", "dev/tsavorite/locking", "dev/tsavorite/storefunctions", "dev/tsavorite/epoch"]}, + {type: 'category', label: 'Tsavorite - Storage Layer', collapsed: true, items: ["dev/tsavorite/intro", "dev/tsavorite/reviv", "dev/tsavorite/locking", "dev/tsavorite/storefunctions", "dev/tsavorite/epochprotection", "dev/tsavorite/logrecord", "dev/tsavorite/object-allocator"]}, "dev/transactions", "dev/custom-commands", "dev/multi-db",