From 75f20bd9d4080d001489901c405a78e091f4e4ab Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 9 Mar 2025 18:28:26 -0400 Subject: [PATCH 01/56] Streams API Work in progress --- Directory.Build.props | 2 +- libs/server/BTreeIndex/BTreeInternals.cs | 314 ++++++++++++++++++ libs/server/Resp/ByteArrayComparer.cs | 2 +- libs/server/Stream/SessionStreamCache.cs | 59 ++++ libs/server/Stream/Stream.cs | 392 +++++++++++++++++++++++ libs/server/Stream/StreamID.cs | 59 ++++ libs/server/Stream/StreamManager.cs | 220 +++++++++++++ 7 files changed, 1046 insertions(+), 2 deletions(-) create mode 100644 libs/server/BTreeIndex/BTreeInternals.cs create mode 100644 libs/server/Stream/SessionStreamCache.cs create mode 100644 libs/server/Stream/Stream.cs create mode 100644 libs/server/Stream/StreamID.cs create mode 100644 libs/server/Stream/StreamManager.cs diff --git a/Directory.Build.props b/Directory.Build.props index 3b5f21c18c9..a9e99adff29 100644 --- a/Directory.Build.props +++ b/Directory.Build.props @@ -12,7 +12,7 @@ true true true - true + false diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs new file mode 100644 index 00000000000..6e8a68d0583 --- /dev/null +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -0,0 +1,314 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Numerics; +using System.Runtime.InteropServices; +using System.Runtime.Intrinsics.X86; +using Tsavorite.core; + +namespace Garnet.server.BTreeIndex +{ + + public enum BTreeNodeType + { + Internal, + Leaf + } + + /// + /// Represents information stored in a node in the B+tree + /// + [StructLayout(LayoutKind.Explicit)] + public unsafe struct NodeData + { + [FieldOffset(0)] + public Value* values; + [FieldOffset(0)] + public BTreeNode** children; + } + + [StructLayout(LayoutKind.Explicit, Size = sizeof(byte) + sizeof(ulong))] + public struct Value + { + [FieldOffset(0)] + public byte valid; + [FieldOffset(1)] + public ulong address; + + public bool Valid + { + get + { + return valid == 1; + } + set + { + valid = (byte)(value ? 1 : 0); + } + } + + public Value(ulong value) + { + this.valid = 1; + this.address = value; + } + } + + public unsafe struct NodeInfo + { + public BTreeNodeType type; + public int count; + public BTreeNode* next; + public BTreeNode* previous; + public uint validCount; + } + + public unsafe struct BTreeNode + { + public static int PAGE_SIZE = 4096; + public static int KEY_SIZE = 16; // key size in bytes. + public static int METADATA_SIZE = sizeof(NodeInfo) + sizeof(SectorAlignedMemory); + public static int LEAF_CAPACITY = (PAGE_SIZE - METADATA_SIZE) / (KEY_SIZE + sizeof(Value)); + public static int INTERNAL_CAPACITY = (PAGE_SIZE - METADATA_SIZE - sizeof(BTreeNode*)) / (KEY_SIZE + sizeof(IntPtr*)); + + public NodeInfo info; + public byte* keys; + public NodeData data; // data in the node + // public IntPtr memoryBlock; // pointer to the memory block + public SectorAlignedMemory memoryHandle; + + /// + /// Allocates memory for a node + /// + /// type of node to allocate memory for + public void Initialize(BTreeNodeType type, SectorAlignedBufferPool bufferPool) + { + // assume this is called after memory has been allocated and memoryBlock is set (it is the first field) + // we are only assigning different parts of the memory to different fields + var startAddr = (byte*)memoryHandle.aligned_pointer; + info.type = type; + info.count = 0; + info.next = null; + info.previous = null; + info.validCount = 0; + + var baseAddress = startAddr + sizeof(NodeInfo); + keys = (byte*)baseAddress; + + int capacity = type == BTreeNodeType.Leaf ? LEAF_CAPACITY : INTERNAL_CAPACITY; + byte* dataAddress = keys + (capacity * KEY_SIZE); + if (type == BTreeNodeType.Leaf) + { + data.values = (Value*)dataAddress; + } + else + { + data.children = (BTreeNode**)dataAddress; + } + } + + public byte* GetKey(int index) + { + byte* keyAddress = keys + (index * KEY_SIZE); + return keyAddress; + } + + public void SetKey(int index, byte* keyData) + { + byte* keyAddress = keys + (index * KEY_SIZE); + Buffer.MemoryCopy(keyData, keyAddress, KEY_SIZE, KEY_SIZE); + } + + public void SetChild(int index, BTreeNode* child) + { + data.children[index] = child; + } + + public BTreeNode* GetChild(int index) + { + return data.children[index]; + } + + public void SetValue(int index, Value value) + { + data.values[index] = value; + } + + public Value GetValue(int index) + { + return data.values[index]; + } + + public void SetValueValid(int index, bool valid) + { + data.values[index].Valid = valid; + } + + public void InsertTombstone(int index) + { + data.values[index].Valid = false; + } + + /// + /// Returns the index of the first key greater than the given key + /// + /// + /// + public int UpperBound(byte* key) + { + if (info.count == 0) + { + return 0; + } + int left = 0, right = info.count - 1; + while (left <= right) + { + var mid = left + (right - left) / 2; + byte* midKey = GetKey(mid); + int cmp = Compare(key, midKey); + if (cmp < 0) + { + right = mid - 1; + } + else + { + left = mid + 1; + } + } + return left; + } + + /// + /// Returns the index of the first key less than the given key + /// + /// + /// + public int LowerBound(byte* key) + { + if (info.count == 0) + { + return 0; + } + int left = 0, right = info.count - 1; + while (left <= right) + { + var mid = left + (right - left) / 2; + byte* midKey = GetKey(mid); + int cmp = Compare(midKey, key); + if (cmp == 0) + { + return mid; + } + else if (cmp < 0) + { + left = mid + 1; + } + else + { + right = mid - 1; + } + } + return left; + } + + /// + /// Upgrades a leaf node to an internal node + /// + public void UpgradeToInternal() + { + info.type = BTreeNodeType.Internal; + data.children = (BTreeNode**)(keys + (INTERNAL_CAPACITY * KEY_SIZE)); // should be keys + Internal capacity? + } + + /// + /// Compares two keys + /// + /// + /// + /// -1 if key1 is less than key2; 0 if key1 == key2; 1 if key1 > key2 + public static int Compare(byte* key1, byte* key2) + { + + if (Sse2.IsSupported) + { + var v1 = Sse2.LoadVector128(key1); + var v2 = Sse2.LoadVector128(key2); + + var mask = Sse2.MoveMask(Sse2.CompareEqual(v1, v2)); + + if (mask != 0xFFFF) // Not all bytes are equal + { + // Find the index of the first differing byte + int index = BitOperations.TrailingZeroCount(~mask); // Invert mask to find first zero (differing byte) + return key1[index] < key2[index] ? -1 : 1; + } + + return 0; // Arrays are equal + } + else + { + return new Span(key1, KEY_SIZE).SequenceCompareTo(new Span(key2, KEY_SIZE)); + } + } + + public void Deallocate() + { + // if (memoryBlock != IntPtr.Zero) + // { + // Marshal.FreeHGlobal(memoryBlock); + // memoryBlock = IntPtr.Zero; + + // // After freeing the memory, explicitly set pointers to null to avoid dangling pointers. + // // info = null; + // keys = null; + // data.values = null; // Only necessary if data.values or data.children was separately allocated + // data.children = null; + // } + + // info = null; + // keys = null; + // data.values = null; + // data.children = null; + + if(memoryHandle.aligned_pointer != null) + { + keys = null; + data.values = null; + data.children = null; + info.next = null; + memoryHandle.Return(); + } + } + } + + /// + /// Statistics about the B+Tree + /// + public struct BTreeStats + { + // general index stats + public int depth; + public ulong numLeafNodes; + public ulong numInternalNodes; + + // workload specific stats + public long totalInserts; // cumulative number of inserts to the index + public long totalDeletes; // cumulative number of deletes to the index + public ulong totalFastInserts; // cumulative number of fast inserts to the index + public long numKeys; // number of keys currently indexed + public ulong numValidKeys; // number of keys that are not tombstoned + + public BTreeStats() + { + depth = 0; + numLeafNodes = 0; + numInternalNodes = 0; + totalInserts = 0; + totalDeletes = 0; + totalFastInserts = 0; + numKeys = 0; + numValidKeys = 0; + } + } +} \ No newline at end of file diff --git a/libs/server/Resp/ByteArrayComparer.cs b/libs/server/Resp/ByteArrayComparer.cs index b7266527597..e5c500bb263 100644 --- a/libs/server/Resp/ByteArrayComparer.cs +++ b/libs/server/Resp/ByteArrayComparer.cs @@ -22,7 +22,7 @@ public sealed class ByteArrayComparer : IEqualityComparer public bool Equals(byte[] left, byte[] right) => new ReadOnlySpan(left).SequenceEqual(new ReadOnlySpan(right)); - private ByteArrayComparer() { } + public ByteArrayComparer() { } /// public unsafe int GetHashCode(byte[] key) diff --git a/libs/server/Stream/SessionStreamCache.cs b/libs/server/Stream/SessionStreamCache.cs new file mode 100644 index 00000000000..94f18bb3c0c --- /dev/null +++ b/libs/server/Stream/SessionStreamCache.cs @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; + +namespace Garnet.server +{ + internal struct SessionStreamCache + { + const int DefaultCacheSize = 16; + readonly Dictionary streamCache = new Dictionary(DefaultCacheSize, new ByteArrayComparer()); + readonly byte[][] streamKeysCache = new byte[DefaultCacheSize][]; + int cachedStreamsCount = 0; + int front = 0; + + public SessionStreamCache() + { } + + /// + /// Lookup a stream in the cahce. Since the cache is expected to be small, we can sequentially scan. + /// + /// name of stream to lookup + /// stream found from the cache + /// true if stream exists in cache + public bool TryGetStreamFromCache(ReadOnlySpan key, out StreamObject stream) + { + return streamCache.TryGetValue(key.ToArray(), out stream); + } + + /// + /// Add a stream to the cache. If the cache is full, we don't add the stream. + /// + /// name of stream + /// reference to stream object + /// true if successfully added + public bool TryAddStreamToCache(byte[] key, StreamObject stream) + { + if (cachedStreamsCount < DefaultCacheSize) + { + streamCache.Add(key, stream); + // add to circular array and update front + streamKeysCache[front] = key; + front = (front + 1) % DefaultCacheSize; + cachedStreamsCount++; + return true; + } + + streamCache.Remove(streamKeysCache[front]); + streamCache.Add(key, stream); + // add to circular array where we removed the oldest stream + streamKeysCache[front] = key; + front = (front + 1) % DefaultCacheSize; + // we don't need to update cachedStreamsCount since we added and removed a stream + return true; + + } + } +} \ No newline at end of file diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs new file mode 100644 index 00000000000..38a0f7888b1 --- /dev/null +++ b/libs/server/Stream/Stream.cs @@ -0,0 +1,392 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Tsavorite.core; +using Garnet.server.BTreeIndex; +using Garnet.common; +using System.Threading; +using System.Diagnostics; +using System.Buffers; +using System.Runtime.CompilerServices; + +namespace Garnet.server +{ + public enum XTRIMOpts + { + MAXLEN, + MINID, + NONE + } + + public class StreamObject : IDisposable + { + readonly IDevice device; + readonly TsavoriteLog log; + readonly BPlusTree index; + StreamID lastId; + long totalEntriesAdded; + SingleWriterMultiReaderLock _lock; + private bool _disposed; + + /// + /// Constructor + /// + /// Directory where the log will be stored + /// Page size of the log used for the stream + public StreamObject(string logDir, long pageSize, long memorySize) + { + device = logDir == null ? new NullDevice() : Devices.CreateLogDevice("streamLogs/" + logDir + "/streamLog", preallocateFile: false); + log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize }); + index = new BPlusTree(device.SectorSize); + totalEntriesAdded = 0; + lastId = default; + _lock = new SingleWriterMultiReaderLock(); + } + + /// + /// Increment the stream ID + /// + /// carries the incremented stream id + public void IncrementID(ref GarnetStreamID incrementedID) + { + while (true) + { + var originalMs = lastId.ms; + var originalSeq = lastId.seq; + + if (originalMs == long.MaxValue) + { + incrementedID = default; + return; + } + + var newMs = originalMs; + var newSeq = originalSeq + 1; + + // if seq overflows, increment timestamp and reset seq + if (newSeq == 0) + { + newMs = originalMs + 1; + newSeq = 0; + } + + // Use Interlocked.CompareExchange to ensure atomic update + var updatedMs = Interlocked.CompareExchange(ref lastId.ms, newMs, originalMs); + if (updatedMs == originalMs) + { + // Successfully updated ms, now update seq + Interlocked.Exchange(ref lastId.seq, newSeq); + incrementedID.setMS(newMs); + incrementedID.setSeq(newSeq); + return; + } + + // If we reach here, it means another thread has updated lastId.ms + // Retry the operation + } + } + + /// + /// Generate the next stream ID + /// + /// StreamID generated + public void GenerateNextID(ref GarnetStreamID id) + { + // ulong timestamp = (ulong)DateTimeOffset.UtcNow.ToUnixTimeMilliseconds(); + ulong timestamp = (ulong)Stopwatch.GetTimestamp() / (ulong)(Stopwatch.Frequency / 1000); + + // if this is the first entry or timestamp is greater than last added entry + if (totalEntriesAdded == 0 || timestamp > lastId.ms) + { + id.setMS(timestamp); + id.setSeq(0); + return; + } + // if timestamp is same as last added entry, increment the sequence number + // if seq overflows, increment timestamp and reset the sequence number + IncrementID(ref id); + } + + // TODO: implement this using parseState functions without operating with RespReadUtils + unsafe bool parseIDString(ArgSlice idSlice, ref GarnetStreamID id) + { + // if we have to auto-generate the whole ID + if (*idSlice.ptr == '*' && idSlice.length == 1) + { + GenerateNextID(ref id); + return true; + } + + // we have to parse user-defined ID + // this can be of following formats: + // 1. ts (seq = 0) + // 2. ts-* (auto-generate seq number) + // 3. ts-seq + + // check if last character is a * + if (*(idSlice.ptr + idSlice.length - 1) == '*') + { + // this has to be of format ts-*, so check if '-' is the preceding character + if (*(idSlice.ptr + idSlice.length - 2) != '-') + { + return false; + } + // parse the timestamp + // slice the id to remove the last two characters + var slicedId = new ArgSlice(idSlice.ptr, idSlice.length - 2); + var idEnd = idSlice.ptr + idSlice.length - 2; + if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idEnd)) + { + return false; + } + + // check if timestamp is greater than last added entry + if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + { + return false; + } + else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + { + IncrementID(ref id); + } + else + { + id.setMS(timestamp); + id.setSeq(0); + } + } + else + { + // find index of '-' in the id + int index = -1; + for (int i = 0; i < idSlice.length; i++) + { + if (*(idSlice.ptr + i) == '-') + { + index = i; + break; + } + } + // if '-' is not found, it has to be of format ts + if (index == -1) + { + if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + idSlice.length)) + { + return false; + } + // check if timestamp is greater than last added entry + if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + { + return false; + } + else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + { + IncrementID(ref id); + } + else + { + id.setMS(timestamp); + id.setSeq(0); + } + } + else + { + // parse the timestamp + // slice the id to remove everything after '-' + var slicedId = new ArgSlice(idSlice.ptr, index); + var slicedSeq = new ArgSlice(idSlice.ptr + index + 1, idSlice.length - index - 1); + if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + index)) + { + return false; + } + var seqBegin = idSlice.ptr + index + 1; + var seqEnd = idSlice.ptr + idSlice.length; + if (!RespReadUtils.ReadUlong(out ulong seq, ref seqBegin, seqEnd)) + { + return false; + } + + if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + { + return false; + } + else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + { + if (seq <= lastId.getSeq()) + { + return false; + } + } + id.setMS(timestamp); + id.setSeq(seq); + } + } + + return true; + } + + /// + /// Adds an entry or item to the stream + /// + /// byte array of the entry to store in the stream + /// True if entry is added successfully + public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output) + { + byte* ptr = output.SpanByte.ToPointer(); + var curr = ptr; + var end = curr + output.Length; + MemoryHandle ptrHandle = default; + bool isMemory = false; + byte* tmpPtr = null; + GarnetStreamID id = default; + // take a lock to ensure thread safety + _lock.WriteLock(); + try + { + bool canParseID = parseIDString(idSlice, ref id); + if (!canParseID) + { + while (!RespWriteUtils.WriteError("ERR Syntax", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + + // add the entry to the log + { + bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(GarnetStreamID), numPairs, value, valueLength, out long retAddress); + if (!enqueueInLog) + { + while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + + var streamValue = new Value((ulong)retAddress); + + bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); + // bool added = true; + if (!added) + { + while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + lastId.setMS(id.ms); + lastId.setSeq(id.seq); + + totalEntriesAdded++; + // write back the ID of the entry added + string idString = $"{id.getMS()}-{id.getSeq()}"; + while (!RespWriteUtils.WriteSimpleString(idString, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + } + } + finally + { + // log.Commit(); + + if (isMemory) ptrHandle.Dispose(); + output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); + _lock.WriteUnlock(); + + } + + } + + /// + /// Get current length of the stream (number of entries in the stream) + /// + /// length of stream + public ulong Length() + { + ulong len = 0; + _lock.ReadLock(); + try + { + // get length of the stream from the index excluding tombstones + len = index.ValidCount; + } + finally + { + _lock.ReadUnlock(); + } + return len; + } + + /// + /// Deletes an entry fromt the stream + /// + /// id of the stream entry to delete + /// true if entry was deleted successfully + public unsafe bool DeleteEntry(ArgSlice idSlice) + { + // first parse the idString + if (!parseCompleteID(idSlice, out GarnetStreamID entryID)) + { + return false; + } + bool deleted = false; + // take a lock to delete from the index + _lock.WriteLock(); + try + { + deleted = index.Delete((byte*)Unsafe.AsPointer(ref entryID.idBytes[0])); + } + finally + { + _lock.WriteUnlock(); + } + return deleted; + } + + unsafe bool parseCompleteID(ArgSlice idSlice, out GarnetStreamID streamID) + { + streamID = default; + // complete ID is of the format ts-seq in input where both ts and seq are ulong + // find the index of '-' in the id + int index = -1; + for (int i = 0; i < idSlice.length; i++) + { + if (*(idSlice.ptr + i) == '-') + { + index = i; + break; + } + } + // parse the timestamp + if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + index)) + { + return false; + } + + // after reading the timestamp, the pointer will be at the '-' character + var seqBegin = idSlice.ptr + 1; + // parse the sequence number + if (!RespReadUtils.ReadUlong(out ulong seq, ref seqBegin, idSlice.ptr + idSlice.length - 1)) + { + return false; + } + + streamID.setMS(timestamp); + streamID.setSeq(seq); + return true; + } + + + + /// + public void Dispose() + { + try + { + log.Dispose(); + device.Dispose(); + } + finally + { + + } + } + } +} \ No newline at end of file diff --git a/libs/server/Stream/StreamID.cs b/libs/server/Stream/StreamID.cs new file mode 100644 index 00000000000..05150c02857 --- /dev/null +++ b/libs/server/Stream/StreamID.cs @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Buffers.Binary; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; + +namespace Garnet.server +{ + /// + /// Represents a GarnetStreamID, which is a 128-bit identifier for an entry in a stream. + /// + [StructLayout(LayoutKind.Explicit)] + public unsafe struct StreamID + { + [FieldOffset(0)] + public ulong ms; + [FieldOffset(8)] + public ulong seq; + [FieldOffset(0)] + public fixed byte idBytes[16]; + + public StreamID(ulong ms, ulong seq) + { + BinaryPrimitives.WriteUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.ms), 8), ms); + BinaryPrimitives.WriteUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.seq), 8), seq); + } + public void setMS(ulong ms) + { + BinaryPrimitives.WriteUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.ms), 8), ms); + } + + public void setSeq(ulong seq) + { + BinaryPrimitives.WriteUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.seq), 8), seq); + } + + public ulong getMS() + { + return ms; + } + + public ulong getSeq() + { + return seq; + } + + public unsafe StreamID(byte[] inputBytes) + { + if (inputBytes.Length != 16) + { + throw new ArgumentException("idBytes must be 16 bytes"); + } + + Buffer.MemoryCopy((byte*)Unsafe.AsPointer(ref inputBytes[0]), (byte*)Unsafe.AsPointer(ref idBytes[0]), 16, 16); + } + } +} \ No newline at end of file diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs new file mode 100644 index 00000000000..e89a369b107 --- /dev/null +++ b/libs/server/Stream/StreamManager.cs @@ -0,0 +1,220 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using Garnet.common; +using Tsavorite.core; + +namespace Garnet.server +{ + public sealed class StreamManager : IDisposable + { + private Dictionary streams; + long defPageSize; + long defMemorySize; + + SingleWriterMultiReaderLock _lock = new SingleWriterMultiReaderLock(); + + public StreamManager(long pageSize, long memorySize) + { + streams = new Dictionary(new ByteArrayComparer()); + defPageSize = pageSize; + defMemorySize = memorySize; + } + + /// + /// Add a new entry to the stream + /// + /// key/name of the stream + /// id of the stream entry + /// payload to the stream + /// length of payload to the stream + /// # k-v pairs in the payload + /// + /// key of last stream accessed (for cache) + /// reference to last stream accessed (for cache) + /// Note: Can refactor some of this code to get rid of streamKey and lastStream params. + public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream) + { + // create a copy as we need to store this key in the dictionary + byte[] key = new byte[keySlice.Length]; + fixed (byte* keyPtr = key) + Buffer.MemoryCopy(keySlice.ptr, keyPtr, keySlice.Length, keySlice.Length); + bool foundStream = false; + StreamObject stream; + lastStream = null; + streamKey = null; + _lock.ReadLock(); + try + { + foundStream = streams.TryGetValue(key, out stream); + if (foundStream) + { + stream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + // update last accessed stream key + lastStream = stream; + streamKey = key; + } + } + finally + { + _lock.ReadUnlock(); + } + if (foundStream) + { + return; + } + // take a write lock + _lock.WriteLock(); + try + { + // retry querying the dictionary to see if some other thread has created the stream + foundStream = streams.TryGetValue(key, out stream); + if (!foundStream) + { + // stream was not found with this key so create a new one + StreamObject newStream = new StreamObject(null, defPageSize, defMemorySize); + newStream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + streams.TryAdd(key, newStream); + streamKey = key; + lastStream = newStream; + } + else + { + // we found the stream but it was not the one that we have in cache, so update the cache + stream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + // update last accessed stream key + lastStream = stream; + streamKey = key; + } + } + finally + { + _lock.WriteUnlock(); + } + return; + } + + /// + /// Get the length of a particular stream + /// + /// key of the stream we want to obtain the length + /// length of the stream + public unsafe ulong StreamLength(ArgSlice keySlice) + { + var key = keySlice.ToArray(); + if (streams != null) + { + bool foundStream = streams.TryGetValue(key, out StreamObject stream); + if (foundStream) + { + return stream.Length(); + } + else + { + // return 0 if stream does not exist, as if it was empty + return 0; + } + } + return 0; + } + + /// + /// Perform range scan in a stream + /// + /// key/name of stream + /// start of range + /// end of range + /// threshold to limit scanning + /// + public void StreamRange(ArgSlice keySlice, string start, string end, int count, ref SpanByteAndMemory output) + { + var key = keySlice.ToArray(); + if (streams != null && streams.Count > 0) + { + bool foundStream = streams.TryGetValue(key, out StreamObject stream); + if (foundStream) + { + // stream.ReadRange(start, end, count, ref output); + } + } + } + + /// + /// Delete an entry from a stream + /// + /// key/name of stream to delete + /// id of stream entry to delete + /// last accessed stream in cache + /// + public bool StreamDelete(ArgSlice keySlice, ArgSlice idSlice, out StreamObject lastSeenStream) + { + bool foundStream; + var key = keySlice.ToArray(); + StreamObject stream; + lastSeenStream = null; + if (streams != null) + { + foundStream = streams.TryGetValue(key, out stream); + + if (foundStream) + { + lastSeenStream = stream; + return stream.DeleteEntry(idSlice); + } + } + return false; + } + + /// + /// Trim a stream + /// + /// key/name of stream + /// parameter to trim by + /// option to trim by (length or ID) + /// # valid keys removed + /// + public bool StreamTrim(ArgSlice keySlice, ArgSlice trimValue, XTRIMOpts optType, out ulong validKeysRemoved) + { + bool foundStream; + var key = keySlice.ToArray(); + StreamObject stream; + validKeysRemoved = 0; + if (streams != null) + { + foundStream = streams.TryGetValue(key, out stream); + + if (foundStream) + { + // return stream.Trim(trimValue, optType, out validKeysRemoved); + } + } + // we return true if the stream does not exist (as Resp expects a 0 result) + return true; + } + + /// + public void Dispose() + { + if (streams != null) + { + _lock.WriteLock(); + try + { + foreach (var stream in streams.Values) + { + stream.Dispose(); + } + + streams.Clear(); + } + finally + { + _lock.WriteUnlock(); + } + } + + } + } +} \ No newline at end of file From 73b3f20711d1eadb4f3bb56416bdf1e4626fa3ca Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 9 Mar 2025 18:34:13 -0400 Subject: [PATCH 02/56] Streams - sector aligned bufferpool for index work in progress --- libs/server/BTreeIndex/BTree.cs | 159 +++++++++++ libs/server/BTreeIndex/BTreeDelete.cs | 35 +++ libs/server/BTreeIndex/BTreeInsert.cs | 228 +++++++++++++++ libs/server/BTreeIndex/BTreeLookup.cs | 161 +++++++++++ libs/server/BTreeIndex/BTreeTraverse.cs | 50 ++++ libs/server/BTreeIndex/BTreeTrim.cs | 351 ++++++++++++++++++++++++ libs/server/Stream/Stream.cs | 4 +- playground/BTree/Btree.csproj | 18 ++ playground/BTree/Program.cs | 191 +++++++++++++ 9 files changed, 1195 insertions(+), 2 deletions(-) create mode 100644 libs/server/BTreeIndex/BTree.cs create mode 100644 libs/server/BTreeIndex/BTreeDelete.cs create mode 100644 libs/server/BTreeIndex/BTreeInsert.cs create mode 100644 libs/server/BTreeIndex/BTreeLookup.cs create mode 100644 libs/server/BTreeIndex/BTreeTraverse.cs create mode 100644 libs/server/BTreeIndex/BTreeTrim.cs create mode 100644 playground/BTree/Btree.csproj create mode 100644 playground/BTree/Program.cs diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs new file mode 100644 index 00000000000..b3013874f8f --- /dev/null +++ b/libs/server/BTreeIndex/BTree.cs @@ -0,0 +1,159 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using Tsavorite.core; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + BTreeNode* root; + BTreeNode* head; + BTreeNode* tail; + byte* tailMinKey; + public static readonly int MAX_TREE_DEPTH = 10; // maximum allowed depth of the tree + static int DEFAULT_SPLIT_LEAF_POSITION = (BTreeNode.LEAF_CAPACITY + 1) / 2; // position at which leaf node is split + static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY - 1; // position at which leaf node is split + + BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf + BTreeStats stats; // statistics about the tree + + SectorAlignedBufferPool bufferPool; + + /// + /// Initializes a new instance of the class. + /// + public BTree(uint sectorSize) + { + Console.WriteLine(sectorSize); + bufferPool = new SectorAlignedBufferPool(1, (int)sectorSize); + var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memory = (IntPtr)memoryBlock.aligned_pointer; + root = (BTreeNode*)memory; + root->memoryHandle = memoryBlock; + root->Initialize(BTreeNodeType.Leaf, bufferPool); + head = tail = root; + root->info.next = root->info.previous = null; + root->info.count = 0; + tailMinKey = null; + rootToTailLeaf = new BTreeNode*[MAX_TREE_DEPTH]; + stats = new BTreeStats(); + stats.depth = 1; + stats.numLeafNodes = 1; + } + + /// + /// Frees the memory allocated for a node + /// + /// BTreeNode to free from memory + private void Free(ref BTreeNode* node) + { + // if (node == null || node->info == null) + if (node == null) + { + return; + } + + if (node->info.type == BTreeNodeType.Internal) + { + for (var i = 0; i <= node->info.count; i++) + { + var child = node->GetChild(i); + Free(ref child); + node->SetChild(i, child); // Update the child pointer in the parent node + } + + } + node->Deallocate(); + + // free the pointer to the node + // Marshal.FreeHGlobal((IntPtr)node); + node = null; + } + + /// + /// Deallocates the memory allocated for the B+Tree + /// + public void Deallocate() + { + if (root == null) + return; + Free(ref root); + root = null; + head = null; + tail = null; + Marshal.FreeHGlobal((IntPtr)root); + Marshal.FreeHGlobal((IntPtr)head); + Marshal.FreeHGlobal((IntPtr)tail); + } + + /// + /// Destructor for the B+tree + /// + ~BTree() + { + Deallocate(); + } + + public ulong FastInserts => stats.totalFastInserts; + public ulong LeafCount => stats.numLeafNodes; + public ulong InternalCount => stats.numInternalNodes; + + public ulong ValidCount => StatsValidCount(); + + public long RootValidCount => GetValidCount(root); + + public long TailValidCount => GetValidCount(tail); + + public long Count() + { + return stats.numKeys; + } + public ulong StatsValidCount() + { + return stats.numValidKeys; + } + + public long GetValidCount(BTreeNode* node) + { + return node->info.validCount; + } + + /// + /// Retrieves the first entry in the B+Tree (smallest key) + /// + /// entry fetched + public KeyValuePair First() + { + BTreeNode* leaf = head; + if (leaf == null) + { + return default; + } + byte[] keyBytes = new byte[BTreeNode.KEY_SIZE]; + Buffer.MemoryCopy(leaf->GetKey(0), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + return new KeyValuePair(keyBytes, leaf->GetValue(0)); + } + + /// + /// Retrieves the last entry in the B+Tree (largest key) + /// + /// entry fetched + public KeyValuePair Last() + { + BTreeNode* leaf = tail; + if (leaf == null) + { + return default; + } + byte[] keyBytes = new byte[BTreeNode.KEY_SIZE]; + Buffer.MemoryCopy(leaf->GetKey(leaf->info.count - 1), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + return new KeyValuePair(keyBytes, leaf->GetValue(leaf->info.count - 1)); + } + + } +} \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeDelete.cs b/libs/server/BTreeIndex/BTreeDelete.cs new file mode 100644 index 00000000000..7ca495c72ac --- /dev/null +++ b/libs/server/BTreeIndex/BTreeDelete.cs @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + + +using System; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + /// + /// Delete a key from the B+tree + /// + /// key to delete + /// + public bool Delete(byte* key) + { + BTreeNode* leaf = null; + var nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + + TraverseToLeaf(ref leaf, ref nodesTraversed, key); + var index = leaf->LowerBound(key); + if (index >= leaf->info.count || BTreeNode.Compare(key, leaf->GetKey(index)) != 0) + { + return false; + } + + // insert a tombstone for the delete + leaf->InsertTombstone(index); + leaf->info.validCount--; + stats.numValidKeys--; + return true; + } + } +} \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs new file mode 100644 index 00000000000..4b62ef1b4c0 --- /dev/null +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -0,0 +1,228 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Runtime.InteropServices; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + public bool Insert(byte* key, Value value) + { + BTreeNode* leaf = null; + stats.totalFastInserts++; + stats.totalInserts++; + stats.numKeys++; + stats.numValidKeys++; + leaf = tail; + return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value); + } + public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value) + { + int index; + + // if leaf has space + if (leaf->info.count < BTreeNode.LEAF_CAPACITY) + { + // append to end of leaf node + leaf->SetKey(leaf->info.count, key); + leaf->SetValue(leaf->info.count, value); + leaf->info.count++; + leaf->info.validCount++; + return true; + } + index = leaf->info.count; + // split the leaf node + return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); + } + + public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) + { + var newLeaf = CreateNewLeafNode(ref leaf); + + // count valid keys in the new leaf, starting at splitLeafPosition in the old leaf + uint newLeafValidCount = 0; + for (int i = SPLIT_LEAF_POSITION; i < BTreeNode.LEAF_CAPACITY; i++) + { + if (leaf->data.values[i].Valid) + { + newLeafValidCount++; + } + } + leaf->info.validCount -= newLeafValidCount; + newLeaf->info.validCount = newLeafValidCount; + + // since input will always arrive sorted as timestamp, the new key always goes to the new leaf + var newIndex = index - SPLIT_LEAF_POSITION; + Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + newLeaf->SetKey(newIndex, key); + + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); + newLeaf->SetValue(newIndex, value); + Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + newLeaf->info.validCount++; + + uint validCount = 0; + // the leaf that is split will also be the tail node; so update the tail pointer + if (leaf == tail) + { + tail = newLeaf; + tailMinKey = newLeaf->GetKey(0); + // validCount in internal nodes of the index excludes the validCount of the tail leaf node (optimizing for performance to avoid traversal) + // thus, when we split the tail leaf, we push up the validCount of the leaf that we split to the internal node + validCount = leaf->info.validCount; + } + + // update the parent node with the new key + + return true; + } + + public BTreeNode* CreateNewLeafNode(ref BTreeNode* leafToSplit) + { + BTreeNode* newLeaf = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); + // newLeaf->memoryBlock = (IntPtr)newLeaf; + newLeaf->Initialize(BTreeNodeType.Leaf, bufferPool); + leafToSplit->info.count = SPLIT_LEAF_POSITION; + leafToSplit->info.next = newLeaf; + newLeaf->info.previous = leafToSplit; + newLeaf->info.next = leafToSplit->info.next; + newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; + stats.numLeafNodes++; + return newLeaf; + } + + public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, ref BTreeNode* child, int splitPos, uint newValidCount) + { + int i; + // starts from parent of leaf node that triggered the push-up. + // if the parent has space, insert the key and child pointer, and return. Otherwise, split and cascade up. + for (i = 1; i < stats.depth; i++) + { + var node = nodesTraversed[i]; + var index = node->UpperBound(key); + + if (node->info.count < BTreeNode.INTERNAL_CAPACITY) + { + // TODO: potentially get rid of this as we will also only be appending to end of internal node due to sorted insertions + Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), + (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); + // move all children + for (var j = node->info.count; j > index; j--) + { + node->SetChild(j + 1, node->GetChild(j)); + } + + node->SetKey(index, key); + node->SetChild(index + 1, child); + node->info.count++; + node->info.validCount += newValidCount; + + // insert does not cascade up, so update validCounts in the parent nodes + for (var j = i + 1; j < stats.depth; j++) + { + nodesTraversed[j]->info.validCount += newValidCount; + } + return; + } + + // split internal node + var newNode = SplitInternalNode(ref node, ref nodesTraversed, ref key, ref child, splitPos, index, i); + if (rootToTailLeaf[i] == node && tail != head && BTreeNode.Compare(key, tailMinKey) <= 0) + { + rootToTailLeaf[i] = newNode; + } + child = newNode; + } + // split root + CreateNewRoot(key, child); + } + + public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) + { + BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); + newNode->Initialize(BTreeNodeType.Internal, bufferPool); + stats.numInternalNodes++; + + node->info.count = splitPos; + newNode->info.count = (BTreeNode.INTERNAL_CAPACITY - splitPos); + newNode->info.next = node->info.next; + newNode->info.previous = node; + node->info.next = newNode; + return newNode; + } + + public BTreeNode* SplitInternalNode(ref BTreeNode* nodeToSplit, ref BTreeNode*[] nodesTraversed, ref byte* key, ref BTreeNode* child, int splitPos, int index, int level) + { + var newNode = CreateInternalNode(ref nodeToSplit, splitPos); + + // scan keys from splitPos to get number of valid keys in the new node + uint newValidCount = 0; + for (int i = splitPos; i < BTreeNode.INTERNAL_CAPACITY; i++) + { + if (nodeToSplit->GetChild(i) != null) + { + newValidCount += nodeToSplit->GetChild(i)->info.validCount; + } + } + newNode->info.validCount = newValidCount; + + // we are inserting in sorted order, so child always goes to newNode + Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + newNode->SetKey(index - nodeToSplit->info.count - 1, key); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + newNode->SetChild(index - nodeToSplit->info.count, child); + key = nodeToSplit->GetKey(nodeToSplit->info.count); + + return newNode; + } + + + public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) + { + BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); + leftNode->Initialize(root->info.type, bufferPool); + + // copy the root node to the left node + // Buffer.MemoryCopy(root->info, leftNode->info, BTreeNode.PAGE_SIZE, BTreeNode.PAGE_SIZE); + + // if root is a leaf, upgrade to internal node + if (root->info.type == BTreeNodeType.Leaf) + { + root->UpgradeToInternal(); + } + + root->info.count = 1; + root->SetKey(0, key); + root->SetChild(0, leftNode); + root->SetChild(1, newlySplitNode); + root->info.next = root->info.previous = null; + root->info.validCount = leftNode->info.validCount; + if (newlySplitNode != tail) + { + root->info.validCount += newlySplitNode->info.validCount; + } + newlySplitNode->info.previous = leftNode; + + if (root == head) + { + head = leftNode; + } + if (rootToTailLeaf[stats.depth - 1] == root) + { + if (tail == root) + { + tail = leftNode; + } + rootToTailLeaf[stats.depth - 1] = leftNode; + } + rootToTailLeaf[stats.depth] = root; + stats.depth++; + stats.numInternalNodes++; + } + } +} \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs new file mode 100644 index 00000000000..87a674341f8 --- /dev/null +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -0,0 +1,161 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Runtime.CompilerServices; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + /// + /// Point lookup in the index + /// + /// lookup key + /// + public Value Get(byte* key) + { + BTreeNode* leaf = null; + var nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + TraverseToLeaf(ref leaf, ref nodesTraversed, key); + + var index = leaf->LowerBound(key); + if (index < leaf->info.count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) + { + var value = leaf->GetValue(index); + if (value.Valid) + { + return value; + } + } + return default; + } + + /// + /// Range lookup in the index + /// + /// start key for the range lookup + /// end key for the range lookup + /// address of the start key + /// address of end key + /// list of tombstones + /// limit entries scanned in the range lookup + /// reverse lookup + /// + public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out List tombstones, long limit = -1, bool reverse = false) + { + Debug.Assert(reverse ? + BTreeNode.Compare(start, end) >= 0 : BTreeNode.Compare(start, end) <= 0, + "Start key should be less than or equal to end key"); + int count = 0; + tombstones = new List(); + BTreeNode* startLeaf = null, endLeaf = null; + BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + int startIndex, endIndex; + + // find the leaf node for the start key + TraverseToLeaf(ref startLeaf, ref nodesTraversed, start); + // find the leaf node for the end key + TraverseToLeaf(ref endLeaf, ref nodesTraversed, end); + + if (reverse) + { + // we find the first slot > start and subtract one index to get the start index + startIndex = startLeaf->UpperBound(start) - 1; + startVal = startLeaf->GetValue(startIndex); + + // we find the first value greater than equal to key and that will be the last index + endIndex = endLeaf->LowerBound(end); + endVal = endLeaf->GetValue(endIndex); + } + else + { + // find the first key in the start leaf that is greater than or equal to the start key + startIndex = startLeaf->LowerBound(start); + startVal = startLeaf->GetValue(startIndex); + // find the last key in the end leaf that is less than or equal to the end key + endIndex = endLeaf->UpperBound(end) - 1; + endVal = endLeaf->GetValue(endIndex); + } + + + // now, we iterate over the leaves between startLeaf[startIndex] and endLeaf[endIndex] (inclusive) and collect all tombstones + BTreeNode* leaf = startLeaf; + uint numScanned = 0; + while (leaf != null) + { + int first, last; + bool breakOutOfOuterLoop = false; + if (reverse) + { + // we would like an inverse traversal + first = leaf == startLeaf ? startIndex : leaf->info.count - 1; + last = leaf == endLeaf ? endIndex : 0; + } + else + { + last = leaf == endLeaf ? endIndex : leaf->info.count - 1; + first = leaf == startLeaf ? startIndex : 0; + } + + for (var i = first; ;) + { + numScanned++; + var value = leaf->GetValue(i); + if (!value.Valid) + { + byte[] key = new byte[BTreeNode.KEY_SIZE]; + Buffer.MemoryCopy(leaf->GetKey(i), Unsafe.AsPointer(ref key[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + tombstones.Add(leaf->GetValue(i)); + } + else + { + // entry will be part of result set + count++; + if (limit != -1 && count >= limit) + { + // update address as required + if (reverse) + { + startVal = value; + } + else + { + endVal = value; + } + breakOutOfOuterLoop = true; + break; + } + } + + if (reverse) + { + if (i <= last) + { + break; + } + i--; + } + else + { + if (i >= last) + { + break; + } + i++; + } + } + // if we have reached the endLeaf + if (leaf == endLeaf || breakOutOfOuterLoop) + { + break; + } + + leaf = reverse ? leaf->info.previous : leaf->info.next; + } + return count; + } + } +} \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeTraverse.cs b/libs/server/BTreeIndex/BTreeTraverse.cs new file mode 100644 index 00000000000..a9fe833c7de --- /dev/null +++ b/libs/server/BTreeIndex/BTreeTraverse.cs @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + public byte* TraverseToLeaf(ref BTreeNode* node, ref BTreeNode*[] nodesTraversed, byte* key) + { + byte* leafMax = null; + BTreeNode* child = root; + for (var i = stats.depth - 1; i > 0; --i) + { + node = child; + nodesTraversed[i] = child; + var slot = node->UpperBound(key); + if (slot != node->info.count) + { + leafMax = node->GetKey(slot); + } + child = node->GetChild(slot); + } + node = child; + nodesTraversed[0] = child; + return leafMax; + } + + public byte* TraverseToLeaf(ref BTreeNode* node, ref BTreeNode*[] nodesTraversed, byte* key, out int[] slots) + { + slots = new int[MAX_TREE_DEPTH]; + byte* leafMax = null; + BTreeNode* child = root; + for (var i = stats.depth - 1; i > 0; --i) + { + node = child; + nodesTraversed[i] = child; + var slot = node->UpperBound(key); + slots[i] = slot; + if (slot != node->info.count) + { + leafMax = node->GetKey(slot); + } + child = node->GetChild(slot); + } + node = child; + nodesTraversed[0] = child; + return leafMax; + } + } +} \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs new file mode 100644 index 00000000000..f7c2df687d8 --- /dev/null +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -0,0 +1,351 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + underflowingNodes = 0; + BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + BTreeNode* leaf = null; + validKeysRemoved = 0; + numLeavesDeleted = 0; + // first find the leaf node that could contain the key + TraverseToLeaf(ref leaf, ref nodesTraversed, key, out int[] internalSlots); + + // find the index for the key in the leaf node. Note: this would return the index of the first key greater than or equal to the given key + var index = leaf->LowerBound(key); + headValidValue = leaf->GetValue(index); + headValidKey = new byte[BTreeNode.KEY_SIZE]; + var headValidKeyPtr = leaf->GetKey(index); + Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + + // TODO: shift entries in current leaf... for now, simply insert tombstones until this point + for (var i = 0; i < index; i++) + { + leaf->SetValueValid(i, false); + leaf->info.validCount--; + } + + if (leaf == head) + { + // we are already at head so we don't need to do any other traversals + numLeavesDeleted = 0; + return; + } + + // we will now traverse the leaf level of the tree and delete all preceding nodes + BTreeNode* node = leaf->info.previous; + + // # nodes to traverse in the subtree rooted at the leaf's parent (leaf is at nodesTraversed[0]). + // We subtract one since we delete preceding nodes. + var nodesToTraverseInSubtree = internalSlots[1] - 1; + uint deletedValidCount = 0; + + while (node != null) + { + var count = node->info.count; + var validCount = node->info.validCount; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += validCount; + nodesToTraverseInSubtree--; + } + + BTreeNode* prev = node->info.previous; + if (prev == null) + { + // should have reached the head, so do a sanity check + Debug.Assert(node == head); + } + + // update stats + stats.numLeafNodes--; + stats.numKeys -= count; + stats.numValidKeys -= validCount; + validKeysRemoved += validCount; + + // deallocate the node + node->Deallocate(); + Marshal.FreeHGlobal((IntPtr)node); + numLeavesDeleted++; + + // assign node to temp to continue + node = prev; + } + leaf->info.previous = null; + // set leaf as the new head + head = leaf; + + // now we will traverse the internal nodes (except root) and delete all preceding nodes + for (int i = 1; i < stats.depth - 1; i++) + { + // first handle the node in the nodesTraversed + node = nodesTraversed[i]; + var slotOfKey = internalSlots[i]; + + if (slotOfKey > 0) + { + // shift keys and children leftwards until slotOfKey (inclusive) + Buffer.MemoryCopy(node->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, node->keys, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(node->data.children + (slotOfKey - 1) + 1, node->data.children, ((slotOfKey - 1)) * sizeof(BTreeNode*), ((slotOfKey - 1)) * sizeof(BTreeNode*)); + } + + var prev_count = node->info.count; + // update count in node + node->info.count -= slotOfKey; + nodesTraversed[i]->info.validCount -= deletedValidCount; + + if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && node->info.count < BTreeNode.INTERNAL_CAPACITY / 2) + { + // TODO: handle underflow... for now, simply track how many such nodes we may have + underflowingNodes++; + } + + // reset deleted valid count for next level + deletedValidCount = 0; + + // next, handle all preceding internal nodes + node = nodesTraversed[i]->info.previous; + while (node != null) + { + BTreeNode* temp = node->info.previous; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += node->info.validCount; + nodesToTraverseInSubtree--; + } + + node->Deallocate(); + + Marshal.FreeHGlobal((IntPtr)node); + + // update stats + stats.numInternalNodes--; + node = temp; + } + // set the previous of nodesTraversed[i] to null as it is the new head in the level + nodesTraversed[i]->info.previous = null; + + // handle corner case where slotOfKey in the internal node points to the last child => after deletion, only one child remains. + // in this case, delete all parent levels and re-assign root. + if (i + 1 < stats.depth) + { + var nextSlot = internalSlots[i + 1]; + if (nextSlot == nodesTraversed[i + 1]->info.count) + { + BTreeNode* newRoot = nodesTraversed[i]; + var orig_depth = stats.depth; + for (int j = i + 1; j < orig_depth; j++) + { + BTreeNode* curr = nodesTraversed[j]; + while (curr != null) + { + BTreeNode* pre = curr->info.previous; + curr->Deallocate(); + Marshal.FreeHGlobal((IntPtr)curr); + stats.numInternalNodes--; + curr = pre; + } + stats.depth--; + } + root = newRoot; + break; + } + } + } + } + + public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + int underflowingNodes; + TrimByID(key, out underflowingNodes, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + } + + // function to trim the tree up to a given length + // for every internal node, we will scan the children from right to left and check its valid count from the info + // if the valid count is less than the length, we will keep the node and all its children to the right + // if the valid count is greater than or equal to the length, we add this node to the nodesTraversed at the current depth and continue from this child at the next level. + // once we reach the leaf level, we will trim everything to the left, and subsequently traverse the nodesTraversed and perform the same operation at every level. + public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nodesTraversed, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + + var depth = stats.depth - 1; + ulong currentValidCount = 0; + BTreeNode* current = node; + int[] internalSlots = new int[MAX_TREE_DEPTH]; + int underflowingNodes = 0; + validKeysRemoved = 0; + numLeavesDeleted = 0; + headValidKey = new byte[BTreeNode.KEY_SIZE]; + // if the length is greater than the total number of valid keys, we will not trim anything + if (length >= stats.numValidKeys) + { + headValidValue = current->GetValue(0); + Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + return; + } + while (depth > 0) + { + if (current->info.type == BTreeNodeType.Internal) + { + for (var i = current->info.count; i >= 0; i--) + { + // get the child node + BTreeNode* child = current->GetChild(i); + // if adding the child node's valid count wille exceed the length, we will continue on this child. Otherwise, we will keep this node and all its children to the right. + if (currentValidCount + child->info.validCount >= length) + { + nodesTraversed[depth] = current; + internalSlots[depth] = i; + current = child; + break; + } + else + { + currentValidCount += child->info.validCount; + } + } + } + depth--; + } + + // we have reached the leaf level. We will now trim everything to the left of the current node. + headValidValue = current->GetValue(0); + Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + + BTreeNode* leaf = current->info.previous; + // might have to make sure that we are in a leaf node + Debug.Assert(leaf->info.type == BTreeNodeType.Leaf); + + uint deletedValidCount = 0; + var nodesToTraverseInSubtree = internalSlots[depth + 1] - 1; + while (leaf != null) + { + var count = leaf->info.count; + var validCount = leaf->info.validCount; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += validCount; + nodesToTraverseInSubtree--; + } + BTreeNode* prev = leaf->info.previous; + if (prev == null) + { + // should have reached the head, so do a sanity check + Debug.Assert(leaf == head); + } + + // update stats + stats.numLeafNodes--; + stats.numKeys -= count; + stats.numValidKeys -= validCount; + validKeysRemoved += validCount; + + // deallocate the node + leaf->Deallocate(); + Marshal.FreeHGlobal((IntPtr)leaf); + numLeavesDeleted++; + + // assign node to temp to continue + leaf = prev; + } + current->info.previous = null; + // set current as the new head + head = current; + + // now we will traverse the nodesTraversed and delete every node to its left, except the root node + for (int i = 1; i <= stats.depth - 1; i++) + { + var slotOfKey = internalSlots[i]; + BTreeNode* inner; + inner = nodesTraversed[i]; + + if (inner == null) + { + break; + } + + if (slotOfKey > 0) + { + // shift keys and children from slotOfKey to the beginning + Buffer.MemoryCopy(inner->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, inner->keys, (slotOfKey - 1) * BTreeNode.KEY_SIZE, (slotOfKey - 1) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(inner->data.children + slotOfKey, inner->data.children, (slotOfKey) * sizeof(BTreeNode*), (slotOfKey) * sizeof(BTreeNode*)); + } + + var prev_count = inner->info.count; + // update count in node + inner->info.count -= slotOfKey; + + nodesTraversed[i]->info.validCount -= deletedValidCount; + + if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info.count < BTreeNode.INTERNAL_CAPACITY / 2) + { + // TODO: handle underflow... for now, simply track how many such nodes we may have + underflowingNodes++; + } + + // grab all validCounts for nodes that we are deleting. + // subtract from parent's validCount for those we deleted + deletedValidCount = 0; + nodesToTraverseInSubtree = slotOfKey - 1; + inner = inner->info.previous; + while (inner != null && inner != root) + { + BTreeNode* temp = inner->info.previous; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += inner->info.validCount; + nodesToTraverseInSubtree--; + } + inner->Deallocate(); + Marshal.FreeHGlobal((IntPtr)inner); + stats.numInternalNodes--; + inner = temp; + } + nodesTraversed[i]->info.previous = null; + + // check the subsequent level in the tree + // if slotOfKey points to the last child, then all parent levels will be deleted + if (i + 1 < stats.depth) + { + var nextSlot = internalSlots[i + 1]; + if (nextSlot == nodesTraversed[i + 1]->info.count) + { + BTreeNode* newRoot = nodesTraversed[i]; + var orig_depth = stats.depth; + for (int j = i + 1; j < orig_depth; j++) + { + BTreeNode* curr = nodesTraversed[j]; + while (curr != null) + { + BTreeNode* pre = curr->info.previous; + curr->Deallocate(); + Marshal.FreeHGlobal((IntPtr)curr); + stats.numInternalNodes--; + curr = pre; + } + stats.depth--; + } + // now that we have deleted all parent nodes, set root to correct node + root = newRoot; + break; + } + } + } + } + + public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + TrimByLength(ref root, length, ref nodesTraversed, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + } + } +} \ No newline at end of file diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 38a0f7888b1..6d5054e1b53 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -23,7 +23,7 @@ public class StreamObject : IDisposable { readonly IDevice device; readonly TsavoriteLog log; - readonly BPlusTree index; + readonly BTree index; StreamID lastId; long totalEntriesAdded; SingleWriterMultiReaderLock _lock; @@ -38,7 +38,7 @@ public StreamObject(string logDir, long pageSize, long memorySize) { device = logDir == null ? new NullDevice() : Devices.CreateLogDevice("streamLogs/" + logDir + "/streamLog", preallocateFile: false); log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize }); - index = new BPlusTree(device.SectorSize); + index = new BTree(device.SectorSize); totalEntriesAdded = 0; lastId = default; _lock = new SingleWriterMultiReaderLock(); diff --git a/playground/BTree/Btree.csproj b/playground/BTree/Btree.csproj new file mode 100644 index 00000000000..b4678381019 --- /dev/null +++ b/playground/BTree/Btree.csproj @@ -0,0 +1,18 @@ + + + + Exe + net8.0 + enable + enable + + + + + + + + + + + diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs new file mode 100644 index 00000000000..e123816158f --- /dev/null +++ b/playground/BTree/Program.cs @@ -0,0 +1,191 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Diagnostics; +using System.Runtime.CompilerServices; +using Garnet.server; +using Garnet.server.BTreeIndex; +using Tsavorite.core; +class Program +{ + /// + /// Playground for the B+tree index implementation + /// + /// + /// + static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* ptr) + { + pool.Return(*ptr); + } + static unsafe void Main(string[] args) + { + // IntPtr memory = Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE); + + var pool = new SectorAlignedBufferPool(1, 4096); + var memoryBlock = pool.Get(4096); + var memory = (IntPtr)memoryBlock.aligned_pointer; + // SectorAlignedMemory* ptr = (SectorAlignedMemory*)memory; + BTreeNode* node = (BTreeNode*)memory; + node->memoryHandle = memoryBlock; + node->Initialize(BTreeNodeType.Leaf, pool); + something(pool, ptr); + return; + var tree = new BTree(4096); + + ulong N = 1000; + bool verbose = false; + bool sanity = false; + if (args.Length > 0) + { + for (int i = 0; i < args.Length; i++) + { + if (args[i] == "--verb") + { + verbose = true; + } + else if (args[i] == "-s") + { + sanity = true; + } + else if (args[i] == "-N") + { + N = ulong.Parse(args[i + 1]); + break; + } + } + } + GarnetStreamID[] streamIDs = new GarnetStreamID[N]; + long duration = 0; + long dur2 = 0; + for (ulong i = 0; i < N; i++) + { + GarnetStreamID x = new GarnetStreamID(i + 1, 0); + Debug.Assert(x.ms > 0); + streamIDs[i] = x; + } + long start = Stopwatch.GetTimestamp(); + Stopwatch sw = new Stopwatch(); + sw.Start(); + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); + } + sw.Stop(); + dur2 = sw.ElapsedTicks; + duration += Stopwatch.GetTimestamp() - start; + double nanosecondsPerTick = (1_000_000_000.0) / Stopwatch.Frequency; + if (verbose) + { + Console.WriteLine("Insertion done"); + Console.WriteLine(" Number of Fast Inserts = " + tree.FastInserts); + Console.WriteLine("Number of Leaves = " + tree.LeafCount); + Console.WriteLine("Number of Internal Nodes = " + tree.InternalCount); + Console.WriteLine("Time for insertion = " + (double)dur2 * nanosecondsPerTick + " ns"); + } + long insertion_time = (long)(dur2 * nanosecondsPerTick); + sw.Reset(); + + // point lookups + sw.Start(); + for (ulong i = 0; i < N; i++) + { + + var value = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])); + if (sanity) + { + Debug.Assert(value.address == i + 1); + } + } + sw.Stop(); + long query_time = (long)(sw.ElapsedTicks * nanosecondsPerTick); + if (verbose) + { + Console.WriteLine("Time for querying = " + query_time + " ns"); + } + sw.Reset(); + + // forward range query + double[] selectivities = [0.01, 0.05, 0.1]; + long[] range_query_times = new long[selectivities.Length]; + Value[] startVal = new Value[selectivities.Length]; + Value[] endVal = new Value[selectivities.Length]; + List[] list = new List[selectivities.Length]; + for (int i = 0; i < selectivities.Length; i++) + { + double selectivity = selectivities[i]; + ulong startIdx, endIdx; + do + { + // get a random start index from 0 to N + startIdx = (ulong)new Random().Next(0, (int)N); + endIdx = (ulong)(startIdx + (N * selectivity)); + } while (endIdx >= N); + sw.Start(); + // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[startIdx].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[endIdx].idBytes[0]), out startVal[i], out endVal[i], out list[i]); + sw.Stop(); + range_query_times[i] = (long)(sw.ElapsedTicks * nanosecondsPerTick); + if (verbose) + { + Console.WriteLine("Time for range query " + (i + 1) + " = " + range_query_times[i] + " ns"); + } + sw.Reset(); + } + if (verbose) + Console.WriteLine("Range query check passed "); + + // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), out Value startValRev, out Value endValRev, out List<(byte[], Value)> listRev, true); + // Console.WriteLine("list length = " + listRev.Count); + // Console.WriteLine("startValRev = " + startValRev.value + "\t endValRev = " + endValRev.value); + // foreach (var item in listRev) + // { + // Console.WriteLine(item.Item2.value); + // } + // Console.WriteLine("Range query reverse check passed "); + + // now let's delete some keys + sw.Reset(); + int num_deletes = 100; + int num_successfully_deleted = 0; + for (int i = 0; i < num_deletes; i++) + { + // generate a random index to delete + int idx = new Random().Next(0, (int)N); + sw.Start(); + bool val = false; + // bool val = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); + sw.Stop(); + if (val) + { + num_successfully_deleted++; + } + } + long deleteTime = (long)(sw.ElapsedTicks * nanosecondsPerTick); + if (verbose) + { + Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); + Console.WriteLine("Time for deletion = " + deleteTime + " ns"); + } + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); + + // // do a range query to check again + // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List<(byte[], Value)> tombstones); + // Console.WriteLine("list length = " + tombstones.Count); + // foreach (var item in tombstones) + // { + // Console.WriteLine(item.Item2.value); + // } + // Console.WriteLine("Delete check passed "); + + //tree.Trim((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out int overflows); + + //tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[0].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[550].idBytes[0]), out Value startVal12, out Value endVal12, out List<(byte[], Value)> tombstones1); + + // print all times collected in a csv format + Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); + tree.Deallocate(); + Console.WriteLine("All checks passed"); + } +} \ No newline at end of file From b4723ec8d9005718ac32ca1f864680fed7d42196 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 9 Mar 2025 20:37:23 -0400 Subject: [PATCH 03/56] sector aligned bufferpool integration works --- Garnet.sln | 13 ++- libs/common/RespReadUtils.cs | 10 ++ libs/server/BTreeIndex/BTree.cs | 11 +-- libs/server/BTreeIndex/BTreeInsert.cs | 24 +++-- libs/server/BTreeIndex/BTreeInternals.cs | 13 ++- libs/server/Stream/Stream.cs | 112 +++++++++++------------ playground/BTree/Program.cs | 29 +++--- 7 files changed, 125 insertions(+), 87 deletions(-) diff --git a/Garnet.sln b/Garnet.sln index 668e5f3133b..724d4d3fcee 100644 --- a/Garnet.sln +++ b/Garnet.sln @@ -1,4 +1,4 @@ -Microsoft Visual Studio Solution File, Format Version 12.00 +Microsoft Visual Studio Solution File, Format Version 12.00 # Visual Studio Version 17 VisualStudioVersion = 17.0.31808.319 MinimumVisualStudioVersion = 10.0.40219.1 @@ -111,6 +111,8 @@ Project("{9A19103F-16F7-4668-BE54-9A1E7A4F7556}") = "Garnet.resources", "libs\re EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "NoOpModule", "playground\NoOpModule\NoOpModule.csproj", "{D4C9A1A0-7053-F072-21F5-4E0C5827136D}" EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Btree", "playground\BTree\Btree.csproj", "{CE12831B-2805-469E-8208-759DC4B4862C}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -335,6 +337,14 @@ Global {D4C9A1A0-7053-F072-21F5-4E0C5827136D}.Release|Any CPU.Build.0 = Release|Any CPU {D4C9A1A0-7053-F072-21F5-4E0C5827136D}.Release|x64.ActiveCfg = Release|Any CPU {D4C9A1A0-7053-F072-21F5-4E0C5827136D}.Release|x64.Build.0 = Release|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Debug|Any CPU.Build.0 = Debug|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Debug|x64.ActiveCfg = Debug|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Debug|x64.Build.0 = Debug|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Release|Any CPU.ActiveCfg = Release|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Release|Any CPU.Build.0 = Release|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Release|x64.ActiveCfg = Release|Any CPU + {CE12831B-2805-469E-8208-759DC4B4862C}.Release|x64.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE @@ -370,6 +380,7 @@ Global {DF2DD03E-87EE-482A-9FBA-6C8FBC23BDC5} = {697766CD-2046-46D9-958A-0FD3B46C98D4} {A48412B4-FD60-467E-A5D9-F155CAB4F907} = {147FCE31-EC09-4C90-8E4D-37CA87ED18C3} {D4C9A1A0-7053-F072-21F5-4E0C5827136D} = {69A71E2C-00E3-42F3-854E-BE157A24834E} + {CE12831B-2805-469E-8208-759DC4B4862C} = {69A71E2C-00E3-42F3-854E-BE157A24834E} EndGlobalSection GlobalSection(ExtensibilityGlobals) = postSolution SolutionGuid = {2C02C405-4798-41CA-AF98-61EDFEF6772E} diff --git a/libs/common/RespReadUtils.cs b/libs/common/RespReadUtils.cs index d0c179cb65c..b78e28ee5a7 100644 --- a/libs/common/RespReadUtils.cs +++ b/libs/common/RespReadUtils.cs @@ -630,6 +630,16 @@ public static bool TryReadInt64WithLengthHeader(out long number, ref byte* ptr, return true; } + /// + /// Tries to read a Ulong from the given ASCII-encoded RESP string. + /// Note: this does not check for any length headers and is simply an accessor to TryReadUlong. + /// + /// If parsing was successful, contains the parsed ulong value. + /// The starting position in the RESP string. Will be advanced if parsing is successful. + /// The current end of the RESP string. + /// True if a ulong was successfully parsed. + public static bool ReadUlong(out ulong number, ref byte* ptr, byte* end) => TryReadUInt64(ref ptr, end, out number, out _); + /// /// Read long with length header /// diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index b3013874f8f..de722dce147 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -29,13 +29,12 @@ public unsafe partial class BTree /// public BTree(uint sectorSize) { - Console.WriteLine(sectorSize); bufferPool = new SectorAlignedBufferPool(1, (int)sectorSize); var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memory = (IntPtr)memoryBlock.aligned_pointer; root = (BTreeNode*)memory; - root->memoryHandle = memoryBlock; - root->Initialize(BTreeNodeType.Leaf, bufferPool); + // root->memoryHandle = memoryBlock; + root->Initialize(BTreeNodeType.Leaf, memoryBlock); head = tail = root; root->info.next = root->info.previous = null; root->info.count = 0; @@ -86,9 +85,9 @@ public void Deallocate() root = null; head = null; tail = null; - Marshal.FreeHGlobal((IntPtr)root); - Marshal.FreeHGlobal((IntPtr)head); - Marshal.FreeHGlobal((IntPtr)tail); + // Marshal.FreeHGlobal((IntPtr)root); + // Marshal.FreeHGlobal((IntPtr)head); + // Marshal.FreeHGlobal((IntPtr)tail); } /// diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 4b62ef1b4c0..7033a445ca1 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -82,9 +82,13 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, public BTreeNode* CreateNewLeafNode(ref BTreeNode* leafToSplit) { - BTreeNode* newLeaf = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); + // BTreeNode* newLeaf = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); // newLeaf->memoryBlock = (IntPtr)newLeaf; - newLeaf->Initialize(BTreeNodeType.Leaf, bufferPool); + var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memory = (IntPtr)memoryBlock.aligned_pointer; + BTreeNode* newLeaf = (BTreeNode*)memory; + // newLeaf->memoryHandle = memoryBlock; + newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); leafToSplit->info.count = SPLIT_LEAF_POSITION; leafToSplit->info.next = newLeaf; newLeaf->info.previous = leafToSplit; @@ -142,8 +146,12 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { - BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); - newNode->Initialize(BTreeNodeType.Internal, bufferPool); + // BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); + var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memory = (IntPtr)memoryBlock.aligned_pointer; + BTreeNode* newNode = (BTreeNode*)memory; + // newNode->memoryHandle = memoryBlock; + newNode->Initialize(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; node->info.count = splitPos; @@ -184,8 +192,12 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { - BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); - leftNode->Initialize(root->info.type, bufferPool); + // BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); + var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memory = (IntPtr)memoryBlock.aligned_pointer; + BTreeNode* leftNode = (BTreeNode*)memory; + // leftNode->memoryHandle = memoryBlock; + leftNode->Initialize(root->info.type, memoryBlock); // copy the root node to the left node // Buffer.MemoryCopy(root->info, leftNode->info, BTreeNode.PAGE_SIZE, BTreeNode.PAGE_SIZE); diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 6e8a68d0583..5e5d2f64846 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -68,9 +68,10 @@ public unsafe struct BTreeNode { public static int PAGE_SIZE = 4096; public static int KEY_SIZE = 16; // key size in bytes. + public static int HEADER_SIZE = sizeof(BTreeNode); public static int METADATA_SIZE = sizeof(NodeInfo) + sizeof(SectorAlignedMemory); - public static int LEAF_CAPACITY = (PAGE_SIZE - METADATA_SIZE) / (KEY_SIZE + sizeof(Value)); - public static int INTERNAL_CAPACITY = (PAGE_SIZE - METADATA_SIZE - sizeof(BTreeNode*)) / (KEY_SIZE + sizeof(IntPtr*)); + public static int LEAF_CAPACITY = (PAGE_SIZE - HEADER_SIZE) / (KEY_SIZE + sizeof(Value)); + public static int INTERNAL_CAPACITY = (PAGE_SIZE - HEADER_SIZE - sizeof(BTreeNode*)) / (KEY_SIZE + sizeof(IntPtr*)); public NodeInfo info; public byte* keys; @@ -82,10 +83,11 @@ public unsafe struct BTreeNode /// Allocates memory for a node /// /// type of node to allocate memory for - public void Initialize(BTreeNodeType type, SectorAlignedBufferPool bufferPool) + public void Initialize(BTreeNodeType type, SectorAlignedMemory handle) { // assume this is called after memory has been allocated and memoryBlock is set (it is the first field) // we are only assigning different parts of the memory to different fields + memoryHandle = handle; var startAddr = (byte*)memoryHandle.aligned_pointer; info.type = type; info.count = 0; @@ -93,7 +95,8 @@ public void Initialize(BTreeNodeType type, SectorAlignedBufferPool bufferPool) info.previous = null; info.validCount = 0; - var baseAddress = startAddr + sizeof(NodeInfo); + // var baseAddress = startAddr + sizeof(NodeInfo) + sizeof(SectorAlignedMemory); + var baseAddress = startAddr + HEADER_SIZE; keys = (byte*)baseAddress; int capacity = type == BTreeNodeType.Leaf ? LEAF_CAPACITY : INTERNAL_CAPACITY; @@ -107,7 +110,7 @@ public void Initialize(BTreeNodeType type, SectorAlignedBufferPool bufferPool) data.children = (BTreeNode**)dataAddress; } } - + public byte* GetKey(int index) { byte* keyAddress = keys + (index * KEY_SIZE); diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 6d5054e1b53..4261a1efa1b 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -48,7 +48,7 @@ public StreamObject(string logDir, long pageSize, long memorySize) /// Increment the stream ID /// /// carries the incremented stream id - public void IncrementID(ref GarnetStreamID incrementedID) + public void IncrementID(ref StreamID incrementedID) { while (true) { @@ -91,7 +91,7 @@ public void IncrementID(ref GarnetStreamID incrementedID) /// Generate the next stream ID /// /// StreamID generated - public void GenerateNextID(ref GarnetStreamID id) + public void GenerateNextID(ref StreamID id) { // ulong timestamp = (ulong)DateTimeOffset.UtcNow.ToUnixTimeMilliseconds(); ulong timestamp = (ulong)Stopwatch.GetTimestamp() / (ulong)(Stopwatch.Frequency / 1000); @@ -109,7 +109,7 @@ public void GenerateNextID(ref GarnetStreamID id) } // TODO: implement this using parseState functions without operating with RespReadUtils - unsafe bool parseIDString(ArgSlice idSlice, ref GarnetStreamID id) + unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) { // if we have to auto-generate the whole ID if (*idSlice.ptr == '*' && idSlice.length == 1) @@ -239,58 +239,58 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int MemoryHandle ptrHandle = default; bool isMemory = false; byte* tmpPtr = null; - GarnetStreamID id = default; + StreamID id = default; // take a lock to ensure thread safety - _lock.WriteLock(); - try - { - bool canParseID = parseIDString(idSlice, ref id); - if (!canParseID) - { - while (!RespWriteUtils.WriteError("ERR Syntax", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - return; - } - - // add the entry to the log - { - bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(GarnetStreamID), numPairs, value, valueLength, out long retAddress); - if (!enqueueInLog) - { - while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - return; - } - - var streamValue = new Value((ulong)retAddress); - - bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); - // bool added = true; - if (!added) - { - while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - return; - } - lastId.setMS(id.ms); - lastId.setSeq(id.seq); - - totalEntriesAdded++; - // write back the ID of the entry added - string idString = $"{id.getMS()}-{id.getSeq()}"; - while (!RespWriteUtils.WriteSimpleString(idString, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - } - } - finally - { - // log.Commit(); - - if (isMemory) ptrHandle.Dispose(); - output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); - _lock.WriteUnlock(); - - } + // _lock.WriteLock(); + // try + // { + // bool canParseID = parseIDString(idSlice, ref id); + // if (!canParseID) + // { + // // while (!RespWriteUtils.WriteError("ERR Syntax", ref curr, end)) + // // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // return; + // } + + // // add the entry to the log + // { + // bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); + // if (!enqueueInLog) + // { + // while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // return; + // } + + // var streamValue = new Value((ulong)retAddress); + + // bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); + // // bool added = true; + // if (!added) + // { + // while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // return; + // } + // lastId.setMS(id.ms); + // lastId.setSeq(id.seq); + + // totalEntriesAdded++; + // // write back the ID of the entry added + // string idString = $"{id.getMS()}-{id.getSeq()}"; + // while (!RespWriteUtils.WriteSimpleString(idString, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // } + // } + // finally + // { + // // log.Commit(); + + // if (isMemory) ptrHandle.Dispose(); + // output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); + // _lock.WriteUnlock(); + + // } } @@ -322,7 +322,7 @@ public ulong Length() public unsafe bool DeleteEntry(ArgSlice idSlice) { // first parse the idString - if (!parseCompleteID(idSlice, out GarnetStreamID entryID)) + if (!parseCompleteID(idSlice, out StreamID entryID)) { return false; } @@ -340,7 +340,7 @@ public unsafe bool DeleteEntry(ArgSlice idSlice) return deleted; } - unsafe bool parseCompleteID(ArgSlice idSlice, out GarnetStreamID streamID) + unsafe bool parseCompleteID(ArgSlice idSlice, out StreamID streamID) { streamID = default; // complete ID is of the format ts-seq in input where both ts and seq are ulong diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index e123816158f..9a55b6dead5 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -1,4 +1,4 @@ -// Copyright (c) Microsoft Corporation. +// Copyright (c) Microsoft Corporation. // Licensed under the MIT license. using System.Diagnostics; @@ -19,17 +19,19 @@ static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* } static unsafe void Main(string[] args) { - // IntPtr memory = Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE); + // // IntPtr memory = Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE); - var pool = new SectorAlignedBufferPool(1, 4096); - var memoryBlock = pool.Get(4096); - var memory = (IntPtr)memoryBlock.aligned_pointer; + // var pool = new SectorAlignedBufferPool(1, 4096); + // var memoryBlock = pool.Get(4096); + // var memory = (IntPtr)memoryBlock.aligned_pointer; // SectorAlignedMemory* ptr = (SectorAlignedMemory*)memory; - BTreeNode* node = (BTreeNode*)memory; - node->memoryHandle = memoryBlock; - node->Initialize(BTreeNodeType.Leaf, pool); - something(pool, ptr); - return; + // BTreeNode* node = (BTreeNode*)memory; + // // node->memoryHandle = memoryBlock; + // node->Initialize(BTreeNodeType.Leaf, memoryBlock); + // StreamID sample = new StreamID(1, 0); + // node->SetKey(0, (byte*)Unsafe.AsPointer(ref sample.idBytes[0])); + // // something(pool, ptr); + // return; var tree = new BTree(4096); ulong N = 1000; @@ -54,12 +56,12 @@ static unsafe void Main(string[] args) } } } - GarnetStreamID[] streamIDs = new GarnetStreamID[N]; + StreamID[] streamIDs = new StreamID[N]; long duration = 0; long dur2 = 0; for (ulong i = 0; i < N; i++) { - GarnetStreamID x = new GarnetStreamID(i + 1, 0); + StreamID x = new StreamID(i + 1, 0); Debug.Assert(x.ms > 0); streamIDs[i] = x; } @@ -68,7 +70,8 @@ static unsafe void Main(string[] args) sw.Start(); for (ulong i = 0; i < N; i++) { - tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); + Value val = new Value(i + 1); + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), val); } sw.Stop(); dur2 = sw.ElapsedTicks; From 1068e531a2ee5bfb9c7e0729fe74e43466232194 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 9 Mar 2025 22:22:41 -0400 Subject: [PATCH 04/56] testing code wip --- libs/server/BTreeIndex/BTree.cs | 1 + libs/server/BTreeIndex/BTreeInsert.cs | 50 +++++++++------------------ playground/BTree/Program.cs | 3 +- 3 files changed, 19 insertions(+), 35 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index de722dce147..4a5ba6c4c7e 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -18,6 +18,7 @@ public unsafe partial class BTree public static readonly int MAX_TREE_DEPTH = 10; // maximum allowed depth of the tree static int DEFAULT_SPLIT_LEAF_POSITION = (BTreeNode.LEAF_CAPACITY + 1) / 2; // position at which leaf node is split static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY - 1; // position at which leaf node is split + static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY - 1; // position at which internal node is split BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf BTreeStats stats; // statistics about the tree diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 7033a445ca1..33e753ee06f 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -76,7 +76,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, } // update the parent node with the new key - + PushUpKeyInInternalNode(ref nodesTraversed, newLeaf->GetKey(0), ref newLeaf, SPLIT_INTERNAL_POSITION, validCount); return true; } @@ -90,10 +90,10 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, // newLeaf->memoryHandle = memoryBlock; newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); leafToSplit->info.count = SPLIT_LEAF_POSITION; - leafToSplit->info.next = newLeaf; newLeaf->info.previous = leafToSplit; newLeaf->info.next = leafToSplit->info.next; newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; + leafToSplit->info.next = newLeaf; stats.numLeafNodes++; return newLeaf; } @@ -153,7 +153,6 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, // newNode->memoryHandle = memoryBlock; newNode->Initialize(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; - node->info.count = splitPos; newNode->info.count = (BTreeNode.INTERNAL_CAPACITY - splitPos); newNode->info.next = node->info.next; @@ -195,44 +194,27 @@ public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) // BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memory = (IntPtr)memoryBlock.aligned_pointer; - BTreeNode* leftNode = (BTreeNode*)memory; + BTreeNode* newRoot = (BTreeNode*)memory; // leftNode->memoryHandle = memoryBlock; - leftNode->Initialize(root->info.type, memoryBlock); + newRoot->Initialize(BTreeNodeType.Internal, memoryBlock); - // copy the root node to the left node - // Buffer.MemoryCopy(root->info, leftNode->info, BTreeNode.PAGE_SIZE, BTreeNode.PAGE_SIZE); - - // if root is a leaf, upgrade to internal node - if (root->info.type == BTreeNodeType.Leaf) - { - root->UpgradeToInternal(); - } + // Set the new root's key to the key being pushed up (key from newlySplitNode). + newRoot->info.count = 1; + newRoot->SetKey(0, key); + // Set its children: left child is the old root; right child is the newly split node. + newRoot->SetChild(0, root); + newRoot->SetChild(1, newlySplitNode); - root->info.count = 1; - root->SetKey(0, key); - root->SetChild(0, leftNode); - root->SetChild(1, newlySplitNode); - root->info.next = root->info.previous = null; - root->info.validCount = leftNode->info.validCount; + // Update the valid count (if desired, handle validCount appropriately). + newRoot->info.validCount = root->info.validCount; if (newlySplitNode != tail) { - root->info.validCount += newlySplitNode->info.validCount; + newRoot->info.validCount += newlySplitNode->info.validCount; } - newlySplitNode->info.previous = leftNode; + newRoot->info.next = newRoot->info.previous = null; - if (root == head) - { - head = leftNode; - } - if (rootToTailLeaf[stats.depth - 1] == root) - { - if (tail == root) - { - tail = leftNode; - } - rootToTailLeaf[stats.depth - 1] = leftNode; - } - rootToTailLeaf[stats.depth] = root; + root = newRoot; + rootToTailLeaf[stats.depth] = newRoot; stats.depth++; stats.numInternalNodes++; } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 9a55b6dead5..9c31508c96b 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -34,7 +34,7 @@ static unsafe void Main(string[] args) // return; var tree = new BTree(4096); - ulong N = 1000; + ulong N = 60000; bool verbose = false; bool sanity = false; if (args.Length > 0) @@ -76,6 +76,7 @@ static unsafe void Main(string[] args) sw.Stop(); dur2 = sw.ElapsedTicks; duration += Stopwatch.GetTimestamp() - start; + Console.WriteLine(" Number of Fast Inserts = " + tree.FastInserts); double nanosecondsPerTick = (1_000_000_000.0) / Stopwatch.Frequency; if (verbose) { From 53e204b14347bd5314c3b462cc52a01f827f7889 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Mon, 10 Mar 2025 00:25:54 -0400 Subject: [PATCH 05/56] bug fixed --- libs/server/BTreeIndex/BTree.cs | 4 +- libs/server/BTreeIndex/BTreeInsert.cs | 74 ++++++++++++++++----------- playground/BTree/Program.cs | 22 ++++++-- 3 files changed, 64 insertions(+), 36 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 4a5ba6c4c7e..bcae4d5dcf4 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -17,8 +17,8 @@ public unsafe partial class BTree byte* tailMinKey; public static readonly int MAX_TREE_DEPTH = 10; // maximum allowed depth of the tree static int DEFAULT_SPLIT_LEAF_POSITION = (BTreeNode.LEAF_CAPACITY + 1) / 2; // position at which leaf node is split - static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY - 1; // position at which leaf node is split - static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY - 1; // position at which internal node is split + static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY; // position at which leaf node is split + static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY; // position at which internal node is split BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf BTreeStats stats; // statistics about the tree diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 33e753ee06f..0aa4bbf6dcd 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -41,28 +41,34 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, { var newLeaf = CreateNewLeafNode(ref leaf); - // count valid keys in the new leaf, starting at splitLeafPosition in the old leaf - uint newLeafValidCount = 0; - for (int i = SPLIT_LEAF_POSITION; i < BTreeNode.LEAF_CAPACITY; i++) - { - if (leaf->data.values[i].Valid) - { - newLeafValidCount++; - } - } - leaf->info.validCount -= newLeafValidCount; - newLeaf->info.validCount = newLeafValidCount; - - // since input will always arrive sorted as timestamp, the new key always goes to the new leaf - var newIndex = index - SPLIT_LEAF_POSITION; - Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); - newLeaf->SetKey(newIndex, key); - - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); - newLeaf->SetValue(newIndex, value); - Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); - newLeaf->info.validCount++; + // // count valid keys in the new leaf, starting at splitLeafPosition in the old leaf + // uint newLeafValidCount = 0; + // for (int i = SPLIT_LEAF_POSITION; i < BTreeNode.LEAF_CAPACITY; i++) + // { + // if (leaf->data.values[i].Valid) + // { + // newLeafValidCount++; + // } + // } + // leaf->info.validCount -= newLeafValidCount; + // newLeaf->info.validCount = newLeafValidCount; + + // // since input will always arrive sorted as timestamp, the new key always goes to the new leaf + // var newIndex = index - SPLIT_LEAF_POSITION; + // Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + // newLeaf->SetKey(newIndex, key); + + // Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); + // Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); + // newLeaf->SetValue(newIndex, value); + // Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + // newLeaf->info.validCount++; + + newLeaf->SetKey(0, key); + newLeaf->SetValue(0, value); + newLeaf->info.count = 1; + newLeaf->info.validCount = 1; + uint validCount = 0; // the leaf that is split will also be the tail node; so update the tail pointer @@ -92,7 +98,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, leafToSplit->info.count = SPLIT_LEAF_POSITION; newLeaf->info.previous = leafToSplit; newLeaf->info.next = leafToSplit->info.next; - newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; + newLeaf->info.count = BTreeNode.LEAF_CAPACITY - SPLIT_LEAF_POSITION; leafToSplit->info.next = newLeaf; stats.numLeafNodes++; return newLeaf; @@ -177,13 +183,21 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, newNode->info.validCount = newValidCount; // we are inserting in sorted order, so child always goes to newNode - Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); - newNode->SetKey(index - nodeToSplit->info.count - 1, key); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); - newNode->SetChild(index - nodeToSplit->info.count, child); - key = nodeToSplit->GetKey(nodeToSplit->info.count); + // Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); + // Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + // newNode->SetKey(index - nodeToSplit->info.count - 1, key); + // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); + // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + // newNode->SetChild(index - nodeToSplit->info.count, child); + + newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info.count)); // left child pointer of the new node part + newNode->SetKey(0, key); + newNode->SetChild(1, child); + newNode->info.count = 1; + // key = nodeToSplit->GetKey(nodeToSplit->info.count); + key = newNode->GetKey(0); + + var childvalid = child->info.validCount; return newNode; } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 9c31508c96b..12f402c303f 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -34,7 +34,7 @@ static unsafe void Main(string[] args) // return; var tree = new BTree(4096); - ulong N = 60000; + ulong N = 60500; bool verbose = false; bool sanity = false; if (args.Length > 0) @@ -70,8 +70,19 @@ static unsafe void Main(string[] args) sw.Start(); for (ulong i = 0; i < N; i++) { - Value val = new Value(i + 1); - tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), val); + if(i == 60374) + { + Console.WriteLine("here"); + } + if(i == 43952) + { + Console.WriteLine("here"); + } + if (i == 57315) + { + Console.WriteLine("here"); + } + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); } sw.Stop(); dur2 = sw.ElapsedTicks; @@ -93,7 +104,10 @@ static unsafe void Main(string[] args) sw.Start(); for (ulong i = 0; i < N; i++) { - + if(i==57315) + { + Console.WriteLine("here"); + } var value = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])); if (sanity) { From c64785a5dc801ddcaac27abdf7a9644a20531ec8 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 20 Mar 2025 20:47:07 -0400 Subject: [PATCH 06/56] fixed major bug in btree --- libs/server/BTreeIndex/BTree.cs | 33 ++-- libs/server/BTreeIndex/BTreeInsert.cs | 190 ++++++++++++++++------- libs/server/BTreeIndex/BTreeInternals.cs | 26 +--- playground/BTree/Program.cs | 16 -- 4 files changed, 157 insertions(+), 108 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index bcae4d5dcf4..4fa6c190293 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -50,29 +50,31 @@ public BTree(uint sectorSize) /// Frees the memory allocated for a node /// /// BTreeNode to free from memory - private void Free(ref BTreeNode* node) + private void Free(ref BTreeNode* node, ref List freedHandles) { - // if (node == null || node->info == null) if (node == null) - { return; - } - + + // If this is an internal node, free all its children first if (node->info.type == BTreeNodeType.Internal) { - for (var i = 0; i <= node->info.count; i++) + for (int i = 0; i <= node->info.count; i++) { - var child = node->GetChild(i); - Free(ref child); - node->SetChild(i, child); // Update the child pointer in the parent node + var child = node->data.children[i]; + if (child != null) + { + Free(ref child, ref freedHandles); + node->data.children[i] = null; + } } - } - node->Deallocate(); - // free the pointer to the node - // Marshal.FreeHGlobal((IntPtr)node); - node = null; + + if (node->memoryHandle != null) + { + node->memoryHandle.Return(); + node->memoryHandle = null; + } } /// @@ -82,7 +84,8 @@ public void Deallocate() { if (root == null) return; - Free(ref root); + List freedHandles = new List(); + Free(ref root, ref freedHandles); root = null; head = null; tail = null; diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 0aa4bbf6dcd..b1a95918cb3 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -8,6 +8,7 @@ namespace Garnet.server.BTreeIndex { public unsafe partial class BTree { + public bool Insert(byte* key, Value value) { BTreeNode* leaf = null; @@ -18,22 +19,45 @@ public bool Insert(byte* key, Value value) leaf = tail; return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value); } - public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value) + public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, bool appendToLeaf = false) { int index; + if(appendToLeaf) + { + // if leaf has space + if (leaf->info.count < BTreeNode.LEAF_CAPACITY) + { + // append to end of leaf node + leaf->SetKey(leaf->info.count, key); + leaf->SetValue(leaf->info.count, value); + leaf->info.count++; + leaf->info.validCount++; + return true; + } + index = leaf->info.count; + return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); + } + + // find the index where the key should be inserted + index = leaf->LowerBound(key); + if (index < leaf->info.count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) + { + // insert is actually an update + leaf->SetValue(index, value); + return false; + } - // if leaf has space if (leaf->info.count < BTreeNode.LEAF_CAPACITY) { - // append to end of leaf node - leaf->SetKey(leaf->info.count, key); - leaf->SetValue(leaf->info.count, value); + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info.count - index) * BTreeNode.KEY_SIZE, (leaf->info.count - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info.count - index) * sizeof(Value), (leaf->info.count - index) * sizeof(Value)); + + leaf->SetKey(index, key); + leaf->SetValue(index, value); leaf->info.count++; leaf->info.validCount++; return true; } - index = leaf->info.count; - // split the leaf node return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); } @@ -41,35 +65,48 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, { var newLeaf = CreateNewLeafNode(ref leaf); - // // count valid keys in the new leaf, starting at splitLeafPosition in the old leaf - // uint newLeafValidCount = 0; - // for (int i = SPLIT_LEAF_POSITION; i < BTreeNode.LEAF_CAPACITY; i++) - // { - // if (leaf->data.values[i].Valid) - // { - // newLeafValidCount++; - // } - // } + // scan the keys from splitLeafPos to get the number of valid keys in the new leaf + uint newLeafValidCount = 0; + for (var i = SPLIT_LEAF_POSITION; i < BTreeNode.LEAF_CAPACITY; i++) + { + if (leaf->data.values[i].Valid) + { + newLeafValidCount++; + } + } // leaf->info.validCount -= newLeafValidCount; // newLeaf->info.validCount = newLeafValidCount; - // // since input will always arrive sorted as timestamp, the new key always goes to the new leaf - // var newIndex = index - SPLIT_LEAF_POSITION; - // Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); - // newLeaf->SetKey(newIndex, key); - - // Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); - // Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); - // newLeaf->SetValue(newIndex, value); - // Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); - // newLeaf->info.validCount++; - - newLeaf->SetKey(0, key); - newLeaf->SetValue(0, value); - newLeaf->info.count = 1; - newLeaf->info.validCount = 1; - + // newLeaf->SetKey(0, key); + // newLeaf->SetValue(0, value); + // newLeaf->info.count = 1; + // newLeaf->info.validCount = 1; + // insert the new key to either the old node or the newly created node, based on the index + if (index >= leaf->info.count) + { + // new key goes to the new leaf + var newIndex = index - leaf->info.count; + Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + newLeaf->SetKey(newIndex, key); + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); + newLeaf->SetValue(newIndex, value); + Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + newLeaf->info.validCount++; + } + else + { + Buffer.MemoryCopy(leaf->keys + (leaf->info.count - 1) * BTreeNode.KEY_SIZE, newLeaf->keys, newLeaf->info.count * BTreeNode.KEY_SIZE, newLeaf->info.count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info.count - index - 1) * BTreeNode.KEY_SIZE, (leaf->info.count - index - 1) * BTreeNode.KEY_SIZE); + leaf->SetKey(index, key); + + Buffer.MemoryCopy(leaf->data.values + leaf->info.count - 1, newLeaf->data.values, newLeaf->info.count * sizeof(Value), newLeaf->info.count * sizeof(Value)); + Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info.count - index - 1) * sizeof(Value), (leaf->info.count - index - 1) * sizeof(Value)); + leaf->SetValue(index, value); + leaf->info.validCount++; + } + uint validCount = 0; // the leaf that is split will also be the tail node; so update the tail pointer if (leaf == tail) @@ -116,19 +153,22 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, if (node->info.count < BTreeNode.INTERNAL_CAPACITY) { - // TODO: potentially get rid of this as we will also only be appending to end of internal node due to sorted insertions - Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), - (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); - // move all children - for (var j = node->info.count; j > index; j--) - { - node->SetChild(j + 1, node->GetChild(j)); - } + // // TODO: potentially get rid of this as we will also only be appending to end of internal node due to sorted insertions + // Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), + // (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); + // // move all children + // for (var j = node->info.count; j > index; j--) + // { + // node->SetChild(j + 1, node->GetChild(j)); + // } + + // node->SetKey(index, key); + // node->SetChild(index + 1, child); + // node->info.count++; + // node->info.validCount += newValidCount; - node->SetKey(index, key); - node->SetChild(index + 1, child); - node->info.count++; - node->info.validCount += newValidCount; + // we can insert + InsertToInternalNodeWithinCapacity(ref node, key, ref child, ref nodesTraversed, index, newValidCount); // insert does not cascade up, so update validCounts in the parent nodes for (var j = i + 1; j < stats.depth; j++) @@ -150,6 +190,23 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, CreateNewRoot(key, child); } + public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, ref BTreeNode* child, ref BTreeNode*[] nodesTraversed, int index, uint newValidCount) + { + // move all keys to the right + Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); + // move all children starting from index+1 to the right using a for loop + for (var j = node->info.count; j > index; j--) + { + node->SetChild(j + 1, node->GetChild(j)); + } + + // insert + node->SetKey(index, key); + node->SetChild(index + 1, child); + node->info.count++; + node->info.validCount += newValidCount; + } + public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { // BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); @@ -190,14 +247,43 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); // newNode->SetChild(index - nodeToSplit->info.count, child); - newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info.count)); // left child pointer of the new node part - newNode->SetKey(0, key); - newNode->SetChild(1, child); - newNode->info.count = 1; - // key = nodeToSplit->GetKey(nodeToSplit->info.count); - key = newNode->GetKey(0); + // newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info.count)); // left child pointer of the new node part + // newNode->SetKey(0, key); + // newNode->SetChild(1, child); + // newNode->info.count = 1; + // // key = nodeToSplit->GetKey(nodeToSplit->info.count); + // key = newNode->GetKey(0); + + // var childvalid = child->info.validCount; - var childvalid = child->info.validCount; + if (index > nodeToSplit->info.count) + { + // child goes to newNode + Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + newNode->SetKey(index - nodeToSplit->info.count - 1, key); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + newNode->SetChild(index - nodeToSplit->info.count, child); + key = nodeToSplit->GetKey(nodeToSplit->info.count); + } + else if (index == nodeToSplit->info.count) + { + Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info.count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info.count * BTreeNode.KEY_SIZE, newNode->info.count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children + 1, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + newNode->SetChild(0, child); + } + else + { + // child goes to old node + Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info.count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info.count * BTreeNode.KEY_SIZE, newNode->info.count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info.count - index) * BTreeNode.KEY_SIZE, (nodeToSplit->info.count - index) * BTreeNode.KEY_SIZE); + nodeToSplit->SetKey(index, key); + Buffer.MemoryCopy(nodeToSplit->data.children + nodeToSplit->info.count, newNode->data.children, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + index + 1, nodeToSplit->data.children + index + 2, (nodeToSplit->info.count - index + 1) * sizeof(BTreeNode*), (nodeToSplit->info.count - index + 1) * sizeof(BTreeNode*)); + nodeToSplit->SetChild(index + 1, child); + key = nodeToSplit->GetKey(nodeToSplit->info.count); + } return newNode; } @@ -220,7 +306,7 @@ public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) newRoot->SetChild(1, newlySplitNode); // Update the valid count (if desired, handle validCount appropriately). - newRoot->info.validCount = root->info.validCount; + newRoot->info.validCount = root->info.validCount; if (newlySplitNode != tail) { newRoot->info.validCount += newlySplitNode->info.validCount; diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 5e5d2f64846..ca64c50f31c 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -257,31 +257,7 @@ public static int Compare(byte* key1, byte* key2) public void Deallocate() { - // if (memoryBlock != IntPtr.Zero) - // { - // Marshal.FreeHGlobal(memoryBlock); - // memoryBlock = IntPtr.Zero; - - // // After freeing the memory, explicitly set pointers to null to avoid dangling pointers. - // // info = null; - // keys = null; - // data.values = null; // Only necessary if data.values or data.children was separately allocated - // data.children = null; - // } - - // info = null; - // keys = null; - // data.values = null; - // data.children = null; - - if(memoryHandle.aligned_pointer != null) - { - keys = null; - data.values = null; - data.children = null; - info.next = null; - memoryHandle.Return(); - } + } } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 12f402c303f..0e0d8d84176 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -70,18 +70,6 @@ static unsafe void Main(string[] args) sw.Start(); for (ulong i = 0; i < N; i++) { - if(i == 60374) - { - Console.WriteLine("here"); - } - if(i == 43952) - { - Console.WriteLine("here"); - } - if (i == 57315) - { - Console.WriteLine("here"); - } tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); } sw.Stop(); @@ -104,10 +92,6 @@ static unsafe void Main(string[] args) sw.Start(); for (ulong i = 0; i < N; i++) { - if(i==57315) - { - Console.WriteLine("here"); - } var value = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])); if (sanity) { From 9a6099e2a4f05ac058e528e84cac01e4db200eb7 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Fri, 21 Mar 2025 14:51:02 -0400 Subject: [PATCH 07/56] updates --- libs/server/BTreeIndex/BTree.cs | 9 ++++--- libs/server/BTreeIndex/BTreeInsert.cs | 33 ++++++++++++++++++------ libs/server/BTreeIndex/BTreeInternals.cs | 17 +++++++++++- playground/BTree/Program.cs | 4 ++- 4 files changed, 50 insertions(+), 13 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 4fa6c190293..eed9a0b80d1 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -17,11 +17,11 @@ public unsafe partial class BTree byte* tailMinKey; public static readonly int MAX_TREE_DEPTH = 10; // maximum allowed depth of the tree static int DEFAULT_SPLIT_LEAF_POSITION = (BTreeNode.LEAF_CAPACITY + 1) / 2; // position at which leaf node is split - static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY; // position at which leaf node is split - static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY; // position at which internal node is split + static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY - 1; // position at which leaf node is split + static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY- 1; // position at which internal node is split BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf - BTreeStats stats; // statistics about the tree + public BTreeStats stats; // statistics about the tree SectorAlignedBufferPool bufferPool; @@ -32,6 +32,7 @@ public BTree(uint sectorSize) { bufferPool = new SectorAlignedBufferPool(1, (int)sectorSize); var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + stats.numAllocates = 1; var memory = (IntPtr)memoryBlock.aligned_pointer; root = (BTreeNode*)memory; // root->memoryHandle = memoryBlock; @@ -73,6 +74,7 @@ private void Free(ref BTreeNode* node, ref List freedHandle if (node->memoryHandle != null) { node->memoryHandle.Return(); + stats.numDeallocates++; node->memoryHandle = null; } } @@ -89,6 +91,7 @@ public void Deallocate() root = null; head = null; tail = null; + stats.printStats(); // Marshal.FreeHGlobal((IntPtr)root); // Marshal.FreeHGlobal((IntPtr)head); // Marshal.FreeHGlobal((IntPtr)tail); diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index b1a95918cb3..39d03a99d5c 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -17,7 +17,7 @@ public bool Insert(byte* key, Value value) stats.numKeys++; stats.numValidKeys++; leaf = tail; - return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value); + return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value,true); } public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, bool appendToLeaf = false) { @@ -63,7 +63,20 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) { - var newLeaf = CreateNewLeafNode(ref leaf); + // var newLeaf = CreateNewLeafNode(ref leaf); + var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + stats.numAllocates++; + var memory = (IntPtr)memoryBlock.aligned_pointer; + BTreeNode* newLeaf = (BTreeNode*)memory; + // newLeaf->memoryHandle = memoryBlock; + newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); + + leaf->info.count = SPLIT_LEAF_POSITION; + newLeaf->info.previous = leaf; + newLeaf->info.next = leaf->info.next; + newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; + leaf->info.next = newLeaf; + stats.numLeafNodes++; // scan the keys from splitLeafPos to get the number of valid keys in the new leaf uint newLeafValidCount = 0; @@ -74,8 +87,8 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, newLeafValidCount++; } } - // leaf->info.validCount -= newLeafValidCount; - // newLeaf->info.validCount = newLeafValidCount; + leaf->info.validCount -= newLeafValidCount; + newLeaf->info.validCount = newLeafValidCount; // newLeaf->SetKey(0, key); // newLeaf->SetValue(0, value); @@ -113,6 +126,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, { tail = newLeaf; tailMinKey = newLeaf->GetKey(0); + rootToTailLeaf[0] = newLeaf; // validCount in internal nodes of the index excludes the validCount of the tail leaf node (optimizing for performance to avoid traversal) // thus, when we split the tail leaf, we push up the validCount of the leaf that we split to the internal node validCount = leaf->info.validCount; @@ -121,21 +135,23 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, // update the parent node with the new key PushUpKeyInInternalNode(ref nodesTraversed, newLeaf->GetKey(0), ref newLeaf, SPLIT_INTERNAL_POSITION, validCount); return true; - } + } public BTreeNode* CreateNewLeafNode(ref BTreeNode* leafToSplit) { // BTreeNode* newLeaf = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); // newLeaf->memoryBlock = (IntPtr)newLeaf; var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + stats.numAllocates++; var memory = (IntPtr)memoryBlock.aligned_pointer; BTreeNode* newLeaf = (BTreeNode*)memory; // newLeaf->memoryHandle = memoryBlock; newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); + leafToSplit->info.count = SPLIT_LEAF_POSITION; newLeaf->info.previous = leafToSplit; newLeaf->info.next = leafToSplit->info.next; - newLeaf->info.count = BTreeNode.LEAF_CAPACITY - SPLIT_LEAF_POSITION; + newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; leafToSplit->info.next = newLeaf; stats.numLeafNodes++; return newLeaf; @@ -211,13 +227,14 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r { // BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + stats.numAllocates++; var memory = (IntPtr)memoryBlock.aligned_pointer; BTreeNode* newNode = (BTreeNode*)memory; // newNode->memoryHandle = memoryBlock; newNode->Initialize(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; node->info.count = splitPos; - newNode->info.count = (BTreeNode.INTERNAL_CAPACITY - splitPos); + newNode->info.count = BTreeNode.INTERNAL_CAPACITY - splitPos; newNode->info.next = node->info.next; newNode->info.previous = node; node->info.next = newNode; @@ -293,6 +310,7 @@ public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { // BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + stats.numAllocates++; var memory = (IntPtr)memoryBlock.aligned_pointer; BTreeNode* newRoot = (BTreeNode*)memory; // leftNode->memoryHandle = memoryBlock; @@ -312,7 +330,6 @@ public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) newRoot->info.validCount += newlySplitNode->info.validCount; } newRoot->info.next = newRoot->info.previous = null; - root = newRoot; rootToTailLeaf[stats.depth] = newRoot; stats.depth++; diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index ca64c50f31c..04a86f4665a 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -277,7 +277,8 @@ public struct BTreeStats public ulong totalFastInserts; // cumulative number of fast inserts to the index public long numKeys; // number of keys currently indexed public ulong numValidKeys; // number of keys that are not tombstoned - + public ulong numAllocates; + public ulong numDeallocates; public BTreeStats() { depth = 0; @@ -288,6 +289,20 @@ public BTreeStats() totalFastInserts = 0; numKeys = 0; numValidKeys = 0; + numAllocates = 0; + numDeallocates = 0; + } + + public void printStats() + { + Console.WriteLine($"Depth: {depth}"); + Console.WriteLine($"Number of leaf nodes: {numLeafNodes}"); + Console.WriteLine($"Number of internal nodes: {numInternalNodes}"); + Console.WriteLine($"Total inserts: {totalInserts}"); + Console.WriteLine($"Total deletes: {totalDeletes}"); + Console.WriteLine($"Total fast inserts: {totalFastInserts}"); + Console.WriteLine($"Number of keys: {numKeys}"); + Console.WriteLine($"Number of valid keys: {numValidKeys}"); } } } \ No newline at end of file diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 0e0d8d84176..85d8b79c640 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -34,7 +34,7 @@ static unsafe void Main(string[] args) // return; var tree = new BTree(4096); - ulong N = 60500; + ulong N = 60376; bool verbose = false; bool sanity = false; if (args.Length > 0) @@ -188,6 +188,8 @@ static unsafe void Main(string[] args) // print all times collected in a csv format Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); tree.Deallocate(); + Console.WriteLine("Num allocates = " + tree.stats.numAllocates); + Console.WriteLine("Num deallocates = " + tree.stats.numDeallocates); Console.WriteLine("All checks passed"); } } \ No newline at end of file From edaa74184a9f73856d6b56e1ce006842162da114 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Fri, 21 Mar 2025 22:38:01 -0400 Subject: [PATCH 08/56] fixed memory alloc by reverting to Marshal.AllocHGlobal --- libs/server/BTreeIndex/BTree.cs | 61 ++++--- libs/server/BTreeIndex/BTreeDelete.cs | 4 +- libs/server/BTreeIndex/BTreeInsert.cs | 214 +++++++++++------------ libs/server/BTreeIndex/BTreeInternals.cs | 113 ++++++++---- libs/server/BTreeIndex/BTreeLookup.cs | 8 +- libs/server/BTreeIndex/BTreeTraverse.cs | 4 +- libs/server/BTreeIndex/BTreeTrim.cs | 72 ++++---- libs/server/Stream/Stream.cs | 2 +- playground/BTree/Program.cs | 10 +- 9 files changed, 259 insertions(+), 229 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index eed9a0b80d1..6f68a21be9f 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -17,8 +17,8 @@ public unsafe partial class BTree byte* tailMinKey; public static readonly int MAX_TREE_DEPTH = 10; // maximum allowed depth of the tree static int DEFAULT_SPLIT_LEAF_POSITION = (BTreeNode.LEAF_CAPACITY + 1) / 2; // position at which leaf node is split - static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY - 1; // position at which leaf node is split - static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY- 1; // position at which internal node is split + static int SPLIT_LEAF_POSITION = BTreeNode.LEAF_CAPACITY; // position at which leaf node is split + static int SPLIT_INTERNAL_POSITION = BTreeNode.INTERNAL_CAPACITY; // position at which internal node is split BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf public BTreeStats stats; // statistics about the tree @@ -30,16 +30,17 @@ public unsafe partial class BTree /// public BTree(uint sectorSize) { - bufferPool = new SectorAlignedBufferPool(1, (int)sectorSize); - var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + bufferPool = new SectorAlignedBufferPool(1, BTreeNode.PAGE_SIZE); + // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates = 1; - var memory = (IntPtr)memoryBlock.aligned_pointer; - root = (BTreeNode*)memory; + // root = (BTreeNode*)memory; // root->memoryHandle = memoryBlock; - root->Initialize(BTreeNodeType.Leaf, memoryBlock); + // root->Initialize(BTreeNodeType.Leaf, memoryBlock); + root = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); head = tail = root; - root->info.next = root->info.previous = null; - root->info.count = 0; + root->info->next = root->info->previous = null; + root->info->count = 0; tailMinKey = null; rootToTailLeaf = new BTreeNode*[MAX_TREE_DEPTH]; stats = new BTreeStats(); @@ -51,32 +52,39 @@ public BTree(uint sectorSize) /// Frees the memory allocated for a node /// /// BTreeNode to free from memory - private void Free(ref BTreeNode* node, ref List freedHandles) + private void Free(ref BTreeNode* node) { if (node == null) return; // If this is an internal node, free all its children first - if (node->info.type == BTreeNodeType.Internal) + if (node->info->type == BTreeNodeType.Internal) { - for (int i = 0; i <= node->info.count; i++) + for (int i = 0; i <= node->info->count; i++) { var child = node->data.children[i]; - if (child != null) - { - Free(ref child, ref freedHandles); - node->data.children[i] = null; - } + Free(ref child); + node->data.children[i] = null; } } - + // Free the memory handle if (node->memoryHandle != null) { - node->memoryHandle.Return(); + Marshal.FreeHGlobal((IntPtr)node->memoryHandle); stats.numDeallocates++; - node->memoryHandle = null; + node = null; } + + + + + // if (node->memoryHandle != null) + // { + // node->memoryHandle.Return(); + // stats.numDeallocates++; + // node->memoryHandle = null; + // } } /// @@ -86,12 +94,13 @@ public void Deallocate() { if (root == null) return; - List freedHandles = new List(); - Free(ref root, ref freedHandles); + Free(ref root); + Console.WriteLine("free complete"); + stats.printStats(); root = null; head = null; tail = null; - stats.printStats(); + // Marshal.FreeHGlobal((IntPtr)root); // Marshal.FreeHGlobal((IntPtr)head); // Marshal.FreeHGlobal((IntPtr)tail); @@ -126,7 +135,7 @@ public ulong StatsValidCount() public long GetValidCount(BTreeNode* node) { - return node->info.validCount; + return node->info->validCount; } /// @@ -157,8 +166,8 @@ public KeyValuePair Last() return default; } byte[] keyBytes = new byte[BTreeNode.KEY_SIZE]; - Buffer.MemoryCopy(leaf->GetKey(leaf->info.count - 1), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); - return new KeyValuePair(keyBytes, leaf->GetValue(leaf->info.count - 1)); + Buffer.MemoryCopy(leaf->GetKey(leaf->info->count - 1), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + return new KeyValuePair(keyBytes, leaf->GetValue(leaf->info->count - 1)); } } diff --git a/libs/server/BTreeIndex/BTreeDelete.cs b/libs/server/BTreeIndex/BTreeDelete.cs index 7ca495c72ac..cd6b49f0b60 100644 --- a/libs/server/BTreeIndex/BTreeDelete.cs +++ b/libs/server/BTreeIndex/BTreeDelete.cs @@ -20,14 +20,14 @@ public bool Delete(byte* key) TraverseToLeaf(ref leaf, ref nodesTraversed, key); var index = leaf->LowerBound(key); - if (index >= leaf->info.count || BTreeNode.Compare(key, leaf->GetKey(index)) != 0) + if (index >= leaf->info->count || BTreeNode.Compare(key, leaf->GetKey(index)) != 0) { return false; } // insert a tombstone for the delete leaf->InsertTombstone(index); - leaf->info.validCount--; + leaf->info->validCount--; stats.numValidKeys--; return true; } diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 39d03a99d5c..1d8fa06d938 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Diagnostics; using System.Runtime.InteropServices; namespace Garnet.server.BTreeIndex @@ -25,37 +26,37 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse if(appendToLeaf) { // if leaf has space - if (leaf->info.count < BTreeNode.LEAF_CAPACITY) + if (leaf->info->count < BTreeNode.LEAF_CAPACITY) { // append to end of leaf node - leaf->SetKey(leaf->info.count, key); - leaf->SetValue(leaf->info.count, value); - leaf->info.count++; - leaf->info.validCount++; + leaf->SetKey(leaf->info->count, key); + leaf->SetValue(leaf->info->count, value); + leaf->info->count++; + leaf->info->validCount++; return true; } - index = leaf->info.count; + index = leaf->info->count; return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); } // find the index where the key should be inserted index = leaf->LowerBound(key); - if (index < leaf->info.count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) + if (index < leaf->info->count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) { // insert is actually an update leaf->SetValue(index, value); return false; } - if (leaf->info.count < BTreeNode.LEAF_CAPACITY) + if (leaf->info->count < BTreeNode.LEAF_CAPACITY) { - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info.count - index) * BTreeNode.KEY_SIZE, (leaf->info.count - index) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info.count - index) * sizeof(Value), (leaf->info.count - index) * sizeof(Value)); + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index) * BTreeNode.KEY_SIZE, (leaf->info->count - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info->count - index) * sizeof(Value), (leaf->info->count - index) * sizeof(Value)); leaf->SetKey(index, key); leaf->SetValue(index, value); - leaf->info.count++; - leaf->info.validCount++; + leaf->info->count++; + leaf->info->validCount++; return true; } return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); @@ -64,18 +65,19 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) { // var newLeaf = CreateNewLeafNode(ref leaf); - var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; - var memory = (IntPtr)memoryBlock.aligned_pointer; - BTreeNode* newLeaf = (BTreeNode*)memory; + BTreeNode* newLeaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); // newLeaf->memoryHandle = memoryBlock; - newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); - - leaf->info.count = SPLIT_LEAF_POSITION; - newLeaf->info.previous = leaf; - newLeaf->info.next = leaf->info.next; - newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; - leaf->info.next = newLeaf; + // newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); + Debug.Assert(leaf!=null); + + leaf->info->count = SPLIT_LEAF_POSITION; + newLeaf->info->previous = leaf; + newLeaf->info->next = leaf->info->next; + newLeaf->info->count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; + leaf->info->next = newLeaf; stats.numLeafNodes++; // scan the keys from splitLeafPos to get the number of valid keys in the new leaf @@ -87,37 +89,37 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, newLeafValidCount++; } } - leaf->info.validCount -= newLeafValidCount; - newLeaf->info.validCount = newLeafValidCount; + leaf->info->validCount -= newLeafValidCount; + newLeaf->info->validCount = newLeafValidCount; // newLeaf->SetKey(0, key); // newLeaf->SetValue(0, value); - // newLeaf->info.count = 1; - // newLeaf->info.validCount = 1; + // newLeaf->info->count = 1; + // newLeaf->info->validCount = 1; // insert the new key to either the old node or the newly created node, based on the index - if (index >= leaf->info.count) + if (index >= leaf->info->count) { // new key goes to the new leaf - var newIndex = index - leaf->info.count; - Buffer.MemoryCopy(leaf->keys + leaf->info.count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + var newIndex = index - leaf->info->count; + Buffer.MemoryCopy(leaf->keys + leaf->info->count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); newLeaf->SetKey(newIndex, key); Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->data.values + leaf->info.count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); + Buffer.MemoryCopy(leaf->data.values + leaf->info->count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); newLeaf->SetValue(newIndex, value); Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); - newLeaf->info.validCount++; + newLeaf->info->validCount++; } else { - Buffer.MemoryCopy(leaf->keys + (leaf->info.count - 1) * BTreeNode.KEY_SIZE, newLeaf->keys, newLeaf->info.count * BTreeNode.KEY_SIZE, newLeaf->info.count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info.count - index - 1) * BTreeNode.KEY_SIZE, (leaf->info.count - index - 1) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->keys + (leaf->info->count - 1) * BTreeNode.KEY_SIZE, newLeaf->keys, newLeaf->info->count * BTreeNode.KEY_SIZE, newLeaf->info->count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE, (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); leaf->SetKey(index, key); - Buffer.MemoryCopy(leaf->data.values + leaf->info.count - 1, newLeaf->data.values, newLeaf->info.count * sizeof(Value), newLeaf->info.count * sizeof(Value)); - Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info.count - index - 1) * sizeof(Value), (leaf->info.count - index - 1) * sizeof(Value)); + Buffer.MemoryCopy(leaf->data.values + leaf->info->count - 1, newLeaf->data.values, newLeaf->info->count * sizeof(Value), newLeaf->info->count * sizeof(Value)); + Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info->count - index - 1) * sizeof(Value), (leaf->info->count - index - 1) * sizeof(Value)); leaf->SetValue(index, value); - leaf->info.validCount++; + leaf->info->validCount++; } uint validCount = 0; @@ -129,7 +131,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, rootToTailLeaf[0] = newLeaf; // validCount in internal nodes of the index excludes the validCount of the tail leaf node (optimizing for performance to avoid traversal) // thus, when we split the tail leaf, we push up the validCount of the leaf that we split to the internal node - validCount = leaf->info.validCount; + validCount = leaf->info->validCount; } // update the parent node with the new key @@ -137,26 +139,6 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, return true; } - public BTreeNode* CreateNewLeafNode(ref BTreeNode* leafToSplit) - { - // BTreeNode* newLeaf = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); - // newLeaf->memoryBlock = (IntPtr)newLeaf; - var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - stats.numAllocates++; - var memory = (IntPtr)memoryBlock.aligned_pointer; - BTreeNode* newLeaf = (BTreeNode*)memory; - // newLeaf->memoryHandle = memoryBlock; - newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); - - leafToSplit->info.count = SPLIT_LEAF_POSITION; - newLeaf->info.previous = leafToSplit; - newLeaf->info.next = leafToSplit->info.next; - newLeaf->info.count = BTreeNode.LEAF_CAPACITY + 1 - SPLIT_LEAF_POSITION; - leafToSplit->info.next = newLeaf; - stats.numLeafNodes++; - return newLeaf; - } - public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, ref BTreeNode* child, int splitPos, uint newValidCount) { int i; @@ -167,21 +149,21 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, var node = nodesTraversed[i]; var index = node->UpperBound(key); - if (node->info.count < BTreeNode.INTERNAL_CAPACITY) + if (node->info->count < BTreeNode.INTERNAL_CAPACITY) { // // TODO: potentially get rid of this as we will also only be appending to end of internal node due to sorted insertions // Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), - // (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); + // (node->info->count - index) * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); // // move all children - // for (var j = node->info.count; j > index; j--) + // for (var j = node->info->count; j > index; j--) // { // node->SetChild(j + 1, node->GetChild(j)); // } // node->SetKey(index, key); // node->SetChild(index + 1, child); - // node->info.count++; - // node->info.validCount += newValidCount; + // node->info->count++; + // node->info->validCount += newValidCount; // we can insert InsertToInternalNodeWithinCapacity(ref node, key, ref child, ref nodesTraversed, index, newValidCount); @@ -189,7 +171,7 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, // insert does not cascade up, so update validCounts in the parent nodes for (var j = i + 1; j < stats.depth; j++) { - nodesTraversed[j]->info.validCount += newValidCount; + nodesTraversed[j]->info->validCount += newValidCount; } return; } @@ -209,9 +191,9 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, ref BTreeNode* child, ref BTreeNode*[] nodesTraversed, int index, uint newValidCount) { // move all keys to the right - Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info.count - index) * BTreeNode.KEY_SIZE, (node->info.count - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info->count - index) * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); // move all children starting from index+1 to the right using a for loop - for (var j = node->info.count; j > index; j--) + for (var j = node->info->count; j > index; j--) { node->SetChild(j + 1, node->GetChild(j)); } @@ -219,25 +201,26 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r // insert node->SetKey(index, key); node->SetChild(index + 1, child); - node->info.count++; - node->info.validCount += newValidCount; + node->info->count++; + node->info->validCount += newValidCount; } public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { // BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); - var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; - var memory = (IntPtr)memoryBlock.aligned_pointer; - BTreeNode* newNode = (BTreeNode*)memory; - // newNode->memoryHandle = memoryBlock; - newNode->Initialize(BTreeNodeType.Internal, memoryBlock); + // BTreeNode* newNode = (BTreeNode*)memory; + // // newNode->memoryHandle = memoryBlock; + // newNode->Initialize(BTreeNodeType.Internal, memoryBlock); + BTreeNode* newNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; - node->info.count = splitPos; - newNode->info.count = BTreeNode.INTERNAL_CAPACITY - splitPos; - newNode->info.next = node->info.next; - newNode->info.previous = node; - node->info.next = newNode; + node->info->count = splitPos; + newNode->info->count = BTreeNode.INTERNAL_CAPACITY - splitPos; + newNode->info->next = node->info->next; + newNode->info->previous = node; + node->info->next = newNode; return newNode; } @@ -251,55 +234,55 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r { if (nodeToSplit->GetChild(i) != null) { - newValidCount += nodeToSplit->GetChild(i)->info.validCount; + newValidCount += nodeToSplit->GetChild(i)->info->validCount; } } - newNode->info.validCount = newValidCount; + newNode->info->validCount = newValidCount; // we are inserting in sorted order, so child always goes to newNode - // Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); - // Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); - // newNode->SetKey(index - nodeToSplit->info.count - 1, key); - // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); - // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); - // newNode->SetChild(index - nodeToSplit->info.count, child); - - // newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info.count)); // left child pointer of the new node part + // Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); + // Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + // newNode->SetKey(index - nodeToSplit->info->count - 1, key); + // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*), (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); + // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); + // newNode->SetChild(index - nodeToSplit->info->count, child); + + // newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info->count)); // left child pointer of the new node part // newNode->SetKey(0, key); // newNode->SetChild(1, child); - // newNode->info.count = 1; - // // key = nodeToSplit->GetKey(nodeToSplit->info.count); + // newNode->info->count = 1; + // // key = nodeToSplit->GetKey(nodeToSplit->info->count); // key = newNode->GetKey(0); - // var childvalid = child->info.validCount; + // var childvalid = child->info->validCount; - if (index > nodeToSplit->info.count) + if (index > nodeToSplit->info->count) { // child goes to newNode - Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info.count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info.count - 1) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info.count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); - newNode->SetKey(index - nodeToSplit->info.count - 1, key); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children, (index - nodeToSplit->info.count) * sizeof(BTreeNode*), (index - nodeToSplit->info.count) * sizeof(BTreeNode*)); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info.count, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); - newNode->SetChild(index - nodeToSplit->info.count, child); - key = nodeToSplit->GetKey(nodeToSplit->info.count); + Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + newNode->SetKey(index - nodeToSplit->info->count - 1, key); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*), (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); + newNode->SetChild(index - nodeToSplit->info->count, child); + key = nodeToSplit->GetKey(nodeToSplit->info->count); } - else if (index == nodeToSplit->info.count) + else if (index == nodeToSplit->info->count) { - Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info.count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info.count * BTreeNode.KEY_SIZE, newNode->info.count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info.count, newNode->data.children + 1, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children + 1, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); newNode->SetChild(0, child); } else { // child goes to old node - Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info.count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info.count * BTreeNode.KEY_SIZE, newNode->info.count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info.count - index) * BTreeNode.KEY_SIZE, (nodeToSplit->info.count - index) * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); + Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE, (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); nodeToSplit->SetKey(index, key); - Buffer.MemoryCopy(nodeToSplit->data.children + nodeToSplit->info.count, newNode->data.children, newNode->info.count * sizeof(BTreeNode*), newNode->info.count * sizeof(BTreeNode*)); - Buffer.MemoryCopy(nodeToSplit->data.children + index + 1, nodeToSplit->data.children + index + 2, (nodeToSplit->info.count - index + 1) * sizeof(BTreeNode*), (nodeToSplit->info.count - index + 1) * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + nodeToSplit->info->count, newNode->data.children, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); + Buffer.MemoryCopy(nodeToSplit->data.children + index + 1, nodeToSplit->data.children + index + 2, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*), (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); nodeToSplit->SetChild(index + 1, child); - key = nodeToSplit->GetKey(nodeToSplit->info.count); + key = nodeToSplit->GetKey(nodeToSplit->info->count); } return newNode; @@ -309,27 +292,28 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { // BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); - var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; - var memory = (IntPtr)memoryBlock.aligned_pointer; - BTreeNode* newRoot = (BTreeNode*)memory; - // leftNode->memoryHandle = memoryBlock; - newRoot->Initialize(BTreeNodeType.Internal, memoryBlock); + // BTreeNode* newRoot = (BTreeNode*)memory; + // // leftNode->memoryHandle = memoryBlock; + // newRoot->Initialize(BTreeNodeType.Internal, memoryBlock); + BTreeNode* newRoot = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); // Set the new root's key to the key being pushed up (key from newlySplitNode). - newRoot->info.count = 1; + newRoot->info->count = 1; newRoot->SetKey(0, key); // Set its children: left child is the old root; right child is the newly split node. newRoot->SetChild(0, root); newRoot->SetChild(1, newlySplitNode); // Update the valid count (if desired, handle validCount appropriately). - newRoot->info.validCount = root->info.validCount; + newRoot->info->validCount = root->info->validCount; if (newlySplitNode != tail) { - newRoot->info.validCount += newlySplitNode->info.validCount; + newRoot->info->validCount += newlySplitNode->info->validCount; } - newRoot->info.next = newRoot->info.previous = null; + newRoot->info->next = newRoot->info->previous = null; root = newRoot; rootToTailLeaf[stats.depth] = newRoot; stats.depth++; diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 04a86f4665a..92d5db27092 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -66,50 +66,87 @@ public unsafe struct NodeInfo public unsafe struct BTreeNode { - public static int PAGE_SIZE = 4096; - public static int KEY_SIZE = 16; // key size in bytes. public static int HEADER_SIZE = sizeof(BTreeNode); - public static int METADATA_SIZE = sizeof(NodeInfo) + sizeof(SectorAlignedMemory); - public static int LEAF_CAPACITY = (PAGE_SIZE - HEADER_SIZE) / (KEY_SIZE + sizeof(Value)); - public static int INTERNAL_CAPACITY = (PAGE_SIZE - HEADER_SIZE - sizeof(BTreeNode*)) / (KEY_SIZE + sizeof(IntPtr*)); + public static int PAGE_SIZE = 4096; // This must be increased if you want to store the BTreeNode header in the block. + public static int KEY_SIZE = 16; // key size in bytes. + public static int METADATA_SIZE = sizeof(NodeInfo); + public static int LEAF_CAPACITY = (PAGE_SIZE - HEADER_SIZE - METADATA_SIZE) / (KEY_SIZE + sizeof(Value)); + public static int INTERNAL_CAPACITY = (PAGE_SIZE - HEADER_SIZE - METADATA_SIZE - sizeof(BTreeNode*)) / (KEY_SIZE + sizeof(BTreeNode*)); - public NodeInfo info; + public NodeInfo* info; + public NodeData data; public byte* keys; - public NodeData data; // data in the node - // public IntPtr memoryBlock; // pointer to the memory block - public SectorAlignedMemory memoryHandle; - - /// - /// Allocates memory for a node - /// - /// type of node to allocate memory for - public void Initialize(BTreeNodeType type, SectorAlignedMemory handle) + // public SectorAlignedMemory memoryHandle; + public IntPtr* memoryHandle; + public static BTreeNode* Create(BTreeNodeType type, IntPtr* handle) { - // assume this is called after memory has been allocated and memoryBlock is set (it is the first field) - // we are only assigning different parts of the memory to different fields - memoryHandle = handle; - var startAddr = (byte*)memoryHandle.aligned_pointer; - info.type = type; - info.count = 0; - info.next = null; - info.previous = null; - info.validCount = 0; - - // var baseAddress = startAddr + sizeof(NodeInfo) + sizeof(SectorAlignedMemory); - var baseAddress = startAddr + HEADER_SIZE; - keys = (byte*)baseAddress; - - int capacity = type == BTreeNodeType.Leaf ? LEAF_CAPACITY : INTERNAL_CAPACITY; - byte* dataAddress = keys + (capacity * KEY_SIZE); + // Place the node header at the beginning of the block. + BTreeNode* node = (BTreeNode*)handle; + node->memoryHandle = handle; + + // Define the start of the payload right after the header. + byte* payloadPtr = (byte*)(handle) + HEADER_SIZE; + + // The NodeInfo will be stored at the start of the payload. + node->info = (NodeInfo*)payloadPtr; + node->info->type = type; + node->info->count = 0; + node->info->next = null; + node->info->previous = null; + node->info->validCount = 0; + + // Data for keys follows the Nodeinfo-> + byte* keysPtr = payloadPtr + METADATA_SIZE; + node->keys = keysPtr; + + int capacity = (type == BTreeNodeType.Leaf) ? LEAF_CAPACITY : INTERNAL_CAPACITY; + int keysSize = capacity * KEY_SIZE; + byte* dataSectionPtr = keysPtr + keysSize; + + // Set up NodeData in-place. if (type == BTreeNodeType.Leaf) { - data.values = (Value*)dataAddress; + node->data.values = (Value*)dataSectionPtr; } else { - data.children = (BTreeNode**)dataAddress; + node->data.children = (BTreeNode**)dataSectionPtr; } + + return node; } + + /// + /// Allocates memory for a node + /// + /// type of node to allocate memory for + // public void Initialize(BTreeNodeType type, SectorAlignedMemory handle) + // { + // // assume this is called after memory has been allocated and memoryBlock is set (it is the first field) + // // we are only assigning different parts of the memory to different fields + // // memoryHandle = handle; + // var startAddr = (byte*)memoryHandle.aligned_pointer; + // info->type = type; + // info->count = 0; + // info->next = null; + // info->previous = null; + // info->validCount = 0; + + // // var baseAddress = startAddr + sizeof(NodeInfo) + sizeof(SectorAlignedMemory); + // var baseAddress = startAddr + HEADER_SIZE; + // keys = (byte*)baseAddress; + + // int capacity = type == BTreeNodeType.Leaf ? LEAF_CAPACITY : INTERNAL_CAPACITY; + // byte* dataAddress = keys + (capacity * KEY_SIZE); + // if (type == BTreeNodeType.Leaf) + // { + // data.values = (Value*)dataAddress; + // } + // else + // { + // data.children = (BTreeNode**)dataAddress; + // } + // } public byte* GetKey(int index) { @@ -160,11 +197,11 @@ public void InsertTombstone(int index) /// public int UpperBound(byte* key) { - if (info.count == 0) + if (info->count == 0) { return 0; } - int left = 0, right = info.count - 1; + int left = 0, right = info->count - 1; while (left <= right) { var mid = left + (right - left) / 2; @@ -189,11 +226,11 @@ public int UpperBound(byte* key) /// public int LowerBound(byte* key) { - if (info.count == 0) + if (info->count == 0) { return 0; } - int left = 0, right = info.count - 1; + int left = 0, right = info->count - 1; while (left <= right) { var mid = left + (right - left) / 2; @@ -220,7 +257,7 @@ public int LowerBound(byte* key) /// public void UpgradeToInternal() { - info.type = BTreeNodeType.Internal; + info->type = BTreeNodeType.Internal; data.children = (BTreeNode**)(keys + (INTERNAL_CAPACITY * KEY_SIZE)); // should be keys + Internal capacity? } diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs index 87a674341f8..a037cc7b616 100644 --- a/libs/server/BTreeIndex/BTreeLookup.cs +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -22,7 +22,7 @@ public Value Get(byte* key) TraverseToLeaf(ref leaf, ref nodesTraversed, key); var index = leaf->LowerBound(key); - if (index < leaf->info.count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) + if (index < leaf->info->count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) { var value = leaf->GetValue(index); if (value.Valid) @@ -91,12 +91,12 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out if (reverse) { // we would like an inverse traversal - first = leaf == startLeaf ? startIndex : leaf->info.count - 1; + first = leaf == startLeaf ? startIndex : leaf->info->count - 1; last = leaf == endLeaf ? endIndex : 0; } else { - last = leaf == endLeaf ? endIndex : leaf->info.count - 1; + last = leaf == endLeaf ? endIndex : leaf->info->count - 1; first = leaf == startLeaf ? startIndex : 0; } @@ -153,7 +153,7 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out break; } - leaf = reverse ? leaf->info.previous : leaf->info.next; + leaf = reverse ? leaf->info->previous : leaf->info->next; } return count; } diff --git a/libs/server/BTreeIndex/BTreeTraverse.cs b/libs/server/BTreeIndex/BTreeTraverse.cs index a9fe833c7de..8c9d482045f 100644 --- a/libs/server/BTreeIndex/BTreeTraverse.cs +++ b/libs/server/BTreeIndex/BTreeTraverse.cs @@ -14,7 +14,7 @@ public unsafe partial class BTree node = child; nodesTraversed[i] = child; var slot = node->UpperBound(key); - if (slot != node->info.count) + if (slot != node->info->count) { leafMax = node->GetKey(slot); } @@ -36,7 +36,7 @@ public unsafe partial class BTree nodesTraversed[i] = child; var slot = node->UpperBound(key); slots[i] = slot; - if (slot != node->info.count) + if (slot != node->info->count) { leafMax = node->GetKey(slot); } diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index f7c2df687d8..287cb354328 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -31,7 +31,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe for (var i = 0; i < index; i++) { leaf->SetValueValid(i, false); - leaf->info.validCount--; + leaf->info->validCount--; } if (leaf == head) @@ -42,7 +42,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } // we will now traverse the leaf level of the tree and delete all preceding nodes - BTreeNode* node = leaf->info.previous; + BTreeNode* node = leaf->info->previous; // # nodes to traverse in the subtree rooted at the leaf's parent (leaf is at nodesTraversed[0]). // We subtract one since we delete preceding nodes. @@ -51,15 +51,15 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe while (node != null) { - var count = node->info.count; - var validCount = node->info.validCount; + var count = node->info->count; + var validCount = node->info->validCount; if (nodesToTraverseInSubtree >= 0) { deletedValidCount += validCount; nodesToTraverseInSubtree--; } - BTreeNode* prev = node->info.previous; + BTreeNode* prev = node->info->previous; if (prev == null) { // should have reached the head, so do a sanity check @@ -80,7 +80,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe // assign node to temp to continue node = prev; } - leaf->info.previous = null; + leaf->info->previous = null; // set leaf as the new head head = leaf; @@ -98,12 +98,12 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe Buffer.MemoryCopy(node->data.children + (slotOfKey - 1) + 1, node->data.children, ((slotOfKey - 1)) * sizeof(BTreeNode*), ((slotOfKey - 1)) * sizeof(BTreeNode*)); } - var prev_count = node->info.count; + var prev_count = node->info->count; // update count in node - node->info.count -= slotOfKey; - nodesTraversed[i]->info.validCount -= deletedValidCount; + node->info->count -= slotOfKey; + nodesTraversed[i]->info->validCount -= deletedValidCount; - if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && node->info.count < BTreeNode.INTERNAL_CAPACITY / 2) + if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && node->info->count < BTreeNode.INTERNAL_CAPACITY / 2) { // TODO: handle underflow... for now, simply track how many such nodes we may have underflowingNodes++; @@ -113,13 +113,13 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe deletedValidCount = 0; // next, handle all preceding internal nodes - node = nodesTraversed[i]->info.previous; + node = nodesTraversed[i]->info->previous; while (node != null) { - BTreeNode* temp = node->info.previous; + BTreeNode* temp = node->info->previous; if (nodesToTraverseInSubtree >= 0) { - deletedValidCount += node->info.validCount; + deletedValidCount += node->info->validCount; nodesToTraverseInSubtree--; } @@ -132,14 +132,14 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe node = temp; } // set the previous of nodesTraversed[i] to null as it is the new head in the level - nodesTraversed[i]->info.previous = null; + nodesTraversed[i]->info->previous = null; // handle corner case where slotOfKey in the internal node points to the last child => after deletion, only one child remains. // in this case, delete all parent levels and re-assign root. if (i + 1 < stats.depth) { var nextSlot = internalSlots[i + 1]; - if (nextSlot == nodesTraversed[i + 1]->info.count) + if (nextSlot == nodesTraversed[i + 1]->info->count) { BTreeNode* newRoot = nodesTraversed[i]; var orig_depth = stats.depth; @@ -148,7 +148,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe BTreeNode* curr = nodesTraversed[j]; while (curr != null) { - BTreeNode* pre = curr->info.previous; + BTreeNode* pre = curr->info->previous; curr->Deallocate(); Marshal.FreeHGlobal((IntPtr)curr); stats.numInternalNodes--; @@ -194,14 +194,14 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod } while (depth > 0) { - if (current->info.type == BTreeNodeType.Internal) + if (current->info->type == BTreeNodeType.Internal) { - for (var i = current->info.count; i >= 0; i--) + for (var i = current->info->count; i >= 0; i--) { // get the child node BTreeNode* child = current->GetChild(i); // if adding the child node's valid count wille exceed the length, we will continue on this child. Otherwise, we will keep this node and all its children to the right. - if (currentValidCount + child->info.validCount >= length) + if (currentValidCount + child->info->validCount >= length) { nodesTraversed[depth] = current; internalSlots[depth] = i; @@ -210,7 +210,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod } else { - currentValidCount += child->info.validCount; + currentValidCount += child->info->validCount; } } } @@ -221,22 +221,22 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod headValidValue = current->GetValue(0); Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); - BTreeNode* leaf = current->info.previous; + BTreeNode* leaf = current->info->previous; // might have to make sure that we are in a leaf node - Debug.Assert(leaf->info.type == BTreeNodeType.Leaf); + Debug.Assert(leaf->info->type == BTreeNodeType.Leaf); uint deletedValidCount = 0; var nodesToTraverseInSubtree = internalSlots[depth + 1] - 1; while (leaf != null) { - var count = leaf->info.count; - var validCount = leaf->info.validCount; + var count = leaf->info->count; + var validCount = leaf->info->validCount; if (nodesToTraverseInSubtree >= 0) { deletedValidCount += validCount; nodesToTraverseInSubtree--; } - BTreeNode* prev = leaf->info.previous; + BTreeNode* prev = leaf->info->previous; if (prev == null) { // should have reached the head, so do a sanity check @@ -257,7 +257,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod // assign node to temp to continue leaf = prev; } - current->info.previous = null; + current->info->previous = null; // set current as the new head head = current; @@ -280,13 +280,13 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod Buffer.MemoryCopy(inner->data.children + slotOfKey, inner->data.children, (slotOfKey) * sizeof(BTreeNode*), (slotOfKey) * sizeof(BTreeNode*)); } - var prev_count = inner->info.count; + var prev_count = inner->info->count; // update count in node - inner->info.count -= slotOfKey; + inner->info->count -= slotOfKey; - nodesTraversed[i]->info.validCount -= deletedValidCount; + nodesTraversed[i]->info->validCount -= deletedValidCount; - if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info.count < BTreeNode.INTERNAL_CAPACITY / 2) + if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info->count < BTreeNode.INTERNAL_CAPACITY / 2) { // TODO: handle underflow... for now, simply track how many such nodes we may have underflowingNodes++; @@ -296,13 +296,13 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod // subtract from parent's validCount for those we deleted deletedValidCount = 0; nodesToTraverseInSubtree = slotOfKey - 1; - inner = inner->info.previous; + inner = inner->info->previous; while (inner != null && inner != root) { - BTreeNode* temp = inner->info.previous; + BTreeNode* temp = inner->info->previous; if (nodesToTraverseInSubtree >= 0) { - deletedValidCount += inner->info.validCount; + deletedValidCount += inner->info->validCount; nodesToTraverseInSubtree--; } inner->Deallocate(); @@ -310,14 +310,14 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod stats.numInternalNodes--; inner = temp; } - nodesTraversed[i]->info.previous = null; + nodesTraversed[i]->info->previous = null; // check the subsequent level in the tree // if slotOfKey points to the last child, then all parent levels will be deleted if (i + 1 < stats.depth) { var nextSlot = internalSlots[i + 1]; - if (nextSlot == nodesTraversed[i + 1]->info.count) + if (nextSlot == nodesTraversed[i + 1]->info->count) { BTreeNode* newRoot = nodesTraversed[i]; var orig_depth = stats.depth; @@ -326,7 +326,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod BTreeNode* curr = nodesTraversed[j]; while (curr != null) { - BTreeNode* pre = curr->info.previous; + BTreeNode* pre = curr->info->previous; curr->Deallocate(); Marshal.FreeHGlobal((IntPtr)curr); stats.numInternalNodes--; diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 4261a1efa1b..c844e36b3ba 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -331,7 +331,7 @@ public unsafe bool DeleteEntry(ArgSlice idSlice) _lock.WriteLock(); try { - deleted = index.Delete((byte*)Unsafe.AsPointer(ref entryID.idBytes[0])); + // deleted = index.Delete((byte*)Unsafe.AsPointer(ref entryID.idBytes[0])); } finally { diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 85d8b79c640..28572729502 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -34,7 +34,8 @@ static unsafe void Main(string[] args) // return; var tree = new BTree(4096); - ulong N = 60376; + // ulong N = 999994; + ulong N = 4000000; bool verbose = false; bool sanity = false; if (args.Length > 0) @@ -71,6 +72,8 @@ static unsafe void Main(string[] args) for (ulong i = 0; i < N; i++) { tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); + var value = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])); + Debug.Assert(value.address == i + 1); } sw.Stop(); dur2 = sw.ElapsedTicks; @@ -93,10 +96,7 @@ static unsafe void Main(string[] args) for (ulong i = 0; i < N; i++) { var value = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])); - if (sanity) - { - Debug.Assert(value.address == i + 1); - } + Debug.Assert(value.address == i + 1); } sw.Stop(); long query_time = (long)(sw.ElapsedTicks * nanosecondsPerTick); From 7710e56b81a7cc980dbb73ea7e6ae5bc0d57c30a Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 09:50:33 -0400 Subject: [PATCH 09/56] code cleanup --- libs/server/BTreeIndex/BTree.cs | 10 ----- libs/server/BTreeIndex/BTreeInsert.cs | 57 ++---------------------- libs/server/BTreeIndex/BTreeInternals.cs | 45 ++----------------- libs/server/BTreeIndex/BTreeLookup.cs | 15 +++---- playground/BTree/Program.cs | 31 ++----------- 5 files changed, 18 insertions(+), 140 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 6f68a21be9f..5d05272f2a6 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -34,9 +34,6 @@ public BTree(uint sectorSize) // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates = 1; - // root = (BTreeNode*)memory; - // root->memoryHandle = memoryBlock; - // root->Initialize(BTreeNodeType.Leaf, memoryBlock); root = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); head = tail = root; root->info->next = root->info->previous = null; @@ -76,9 +73,6 @@ private void Free(ref BTreeNode* node) node = null; } - - - // if (node->memoryHandle != null) // { // node->memoryHandle.Return(); @@ -100,10 +94,6 @@ public void Deallocate() root = null; head = null; tail = null; - - // Marshal.FreeHGlobal((IntPtr)root); - // Marshal.FreeHGlobal((IntPtr)head); - // Marshal.FreeHGlobal((IntPtr)tail); } /// diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 1d8fa06d938..68d982dfc6e 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -64,15 +64,11 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) { - // var newLeaf = CreateNewLeafNode(ref leaf); // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; BTreeNode* newLeaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); - // newLeaf->memoryHandle = memoryBlock; - // newLeaf->Initialize(BTreeNodeType.Leaf, memoryBlock); - Debug.Assert(leaf!=null); - + leaf->info->count = SPLIT_LEAF_POSITION; newLeaf->info->previous = leaf; newLeaf->info->next = leaf->info->next; @@ -91,11 +87,6 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, } leaf->info->validCount -= newLeafValidCount; newLeaf->info->validCount = newLeafValidCount; - - // newLeaf->SetKey(0, key); - // newLeaf->SetValue(0, value); - // newLeaf->info->count = 1; - // newLeaf->info->validCount = 1; // insert the new key to either the old node or the newly created node, based on the index if (index >= leaf->info->count) { @@ -151,24 +142,10 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, if (node->info->count < BTreeNode.INTERNAL_CAPACITY) { - // // TODO: potentially get rid of this as we will also only be appending to end of internal node due to sorted insertions - // Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), - // (node->info->count - index) * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); - // // move all children - // for (var j = node->info->count; j > index; j--) - // { - // node->SetChild(j + 1, node->GetChild(j)); - // } - - // node->SetKey(index, key); - // node->SetChild(index + 1, child); - // node->info->count++; - // node->info->validCount += newValidCount; - // we can insert InsertToInternalNodeWithinCapacity(ref node, key, ref child, ref nodesTraversed, index, newValidCount); - // insert does not cascade up, so update validCounts in the parent nodes + // update validCounts in the parent nodes for (var j = i + 1; j < stats.depth; j++) { nodesTraversed[j]->info->validCount += newValidCount; @@ -207,13 +184,9 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { - // BTreeNode* newNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)); // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; - // BTreeNode* newNode = (BTreeNode*)memory; - // // newNode->memoryHandle = memoryBlock; - // newNode->Initialize(BTreeNodeType.Internal, memoryBlock); BTreeNode* newNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; node->info->count = splitPos; @@ -239,23 +212,6 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r } newNode->info->validCount = newValidCount; - // we are inserting in sorted order, so child always goes to newNode - // Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); - // Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); - // newNode->SetKey(index - nodeToSplit->info->count - 1, key); - // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*), (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); - // Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); - // newNode->SetChild(index - nodeToSplit->info->count, child); - - // newNode->SetChild(0, nodeToSplit->GetChild(nodeToSplit->info->count)); // left child pointer of the new node part - // newNode->SetKey(0, key); - // newNode->SetChild(1, child); - // newNode->info->count = 1; - // // key = nodeToSplit->GetKey(nodeToSplit->info->count); - // key = newNode->GetKey(0); - - // var childvalid = child->info->validCount; - if (index > nodeToSplit->info->count) { // child goes to newNode @@ -291,23 +247,18 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { - // BTreeNode* leftNode = (BTreeNode*)Marshal.AllocHGlobal(sizeof(BTreeNode)).ToPointer(); // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); stats.numAllocates++; - // BTreeNode* newRoot = (BTreeNode*)memory; - // // leftNode->memoryHandle = memoryBlock; - // newRoot->Initialize(BTreeNodeType.Internal, memoryBlock); BTreeNode* newRoot = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); - // Set the new root's key to the key being pushed up (key from newlySplitNode). + // Set the new root's key. newRoot->info->count = 1; newRoot->SetKey(0, key); - // Set its children: left child is the old root; right child is the newly split node. + // Set children: left child is the old root; right child is the newly split node. newRoot->SetChild(0, root); newRoot->SetChild(1, newlySplitNode); - // Update the valid count (if desired, handle validCount appropriately). newRoot->info->validCount = root->info->validCount; if (newlySplitNode != tail) { diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 92d5db27092..0203352c83b 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -115,38 +115,6 @@ public unsafe struct BTreeNode return node; } - - /// - /// Allocates memory for a node - /// - /// type of node to allocate memory for - // public void Initialize(BTreeNodeType type, SectorAlignedMemory handle) - // { - // // assume this is called after memory has been allocated and memoryBlock is set (it is the first field) - // // we are only assigning different parts of the memory to different fields - // // memoryHandle = handle; - // var startAddr = (byte*)memoryHandle.aligned_pointer; - // info->type = type; - // info->count = 0; - // info->next = null; - // info->previous = null; - // info->validCount = 0; - - // // var baseAddress = startAddr + sizeof(NodeInfo) + sizeof(SectorAlignedMemory); - // var baseAddress = startAddr + HEADER_SIZE; - // keys = (byte*)baseAddress; - - // int capacity = type == BTreeNodeType.Leaf ? LEAF_CAPACITY : INTERNAL_CAPACITY; - // byte* dataAddress = keys + (capacity * KEY_SIZE); - // if (type == BTreeNodeType.Leaf) - // { - // data.values = (Value*)dataAddress; - // } - // else - // { - // data.children = (BTreeNode**)dataAddress; - // } - // } public byte* GetKey(int index) { @@ -252,15 +220,6 @@ public int LowerBound(byte* key) return left; } - /// - /// Upgrades a leaf node to an internal node - /// - public void UpgradeToInternal() - { - info->type = BTreeNodeType.Internal; - data.children = (BTreeNode**)(keys + (INTERNAL_CAPACITY * KEY_SIZE)); // should be keys + Internal capacity? - } - /// /// Compares two keys /// @@ -294,7 +253,7 @@ public static int Compare(byte* key1, byte* key2) public void Deallocate() { - + // memoryHandle.Dispose(); } } @@ -340,6 +299,8 @@ public void printStats() Console.WriteLine($"Total fast inserts: {totalFastInserts}"); Console.WriteLine($"Number of keys: {numKeys}"); Console.WriteLine($"Number of valid keys: {numValidKeys}"); + Console.WriteLine($"Number of allocates: {numAllocates}"); + Console.WriteLine($"Number of deallocates: {numDeallocates}"); } } } \ No newline at end of file diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs index a037cc7b616..bfeb9290cd5 100644 --- a/libs/server/BTreeIndex/BTreeLookup.cs +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -62,11 +62,11 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out if (reverse) { - // we find the first slot > start and subtract one index to get the start index + // find the first slot > start and subtract one index to get the start index startIndex = startLeaf->UpperBound(start) - 1; startVal = startLeaf->GetValue(startIndex); - // we find the first value greater than equal to key and that will be the last index + // find the first value greater than equal to key and that will be the last index endIndex = endLeaf->LowerBound(end); endVal = endLeaf->GetValue(endIndex); } @@ -80,14 +80,13 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out endVal = endLeaf->GetValue(endIndex); } - - // now, we iterate over the leaves between startLeaf[startIndex] and endLeaf[endIndex] (inclusive) and collect all tombstones + // iterate over the leaves between startLeaf[startIndex] and endLeaf[endIndex] (inclusive) and collect all tombstones BTreeNode* leaf = startLeaf; uint numScanned = 0; while (leaf != null) { int first, last; - bool breakOutOfOuterLoop = false; + bool scanComplete = false; if (reverse) { // we would like an inverse traversal @@ -125,7 +124,7 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out { endVal = value; } - breakOutOfOuterLoop = true; + scanComplete = true; break; } } @@ -147,8 +146,8 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out i++; } } - // if we have reached the endLeaf - if (leaf == endLeaf || breakOutOfOuterLoop) + + if (leaf == endLeaf || scanComplete) { break; } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 28572729502..fa8ac3e27f8 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -19,24 +19,9 @@ static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* } static unsafe void Main(string[] args) { - // // IntPtr memory = Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE); - - // var pool = new SectorAlignedBufferPool(1, 4096); - // var memoryBlock = pool.Get(4096); - // var memory = (IntPtr)memoryBlock.aligned_pointer; - // SectorAlignedMemory* ptr = (SectorAlignedMemory*)memory; - // BTreeNode* node = (BTreeNode*)memory; - // // node->memoryHandle = memoryBlock; - // node->Initialize(BTreeNodeType.Leaf, memoryBlock); - // StreamID sample = new StreamID(1, 0); - // node->SetKey(0, (byte*)Unsafe.AsPointer(ref sample.idBytes[0])); - // // something(pool, ptr); - // return; var tree = new BTree(4096); - - // ulong N = 999994; - ulong N = 4000000; - bool verbose = false; + ulong N = 400000; + bool verbose = true; bool sanity = false; if (args.Length > 0) { @@ -123,7 +108,8 @@ static unsafe void Main(string[] args) endIdx = (ulong)(startIdx + (N * selectivity)); } while (endIdx >= N); sw.Start(); - // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[startIdx].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[endIdx].idBytes[0]), out startVal[i], out endVal[i], out list[i]); + var count = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[startIdx].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[endIdx].idBytes[0]), out startVal[i], out endVal[i], out list[i]); + Debug.Assert(count == (int)(endIdx - startIdx + 1)); sw.Stop(); range_query_times[i] = (long)(sw.ElapsedTicks * nanosecondsPerTick); if (verbose) @@ -135,15 +121,6 @@ static unsafe void Main(string[] args) if (verbose) Console.WriteLine("Range query check passed "); - // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), out Value startValRev, out Value endValRev, out List<(byte[], Value)> listRev, true); - // Console.WriteLine("list length = " + listRev.Count); - // Console.WriteLine("startValRev = " + startValRev.value + "\t endValRev = " + endValRev.value); - // foreach (var item in listRev) - // { - // Console.WriteLine(item.Item2.value); - // } - // Console.WriteLine("Range query reverse check passed "); - // now let's delete some keys sw.Reset(); int num_deletes = 100; From 0129acaa11de6ee5dedf54b2749a035eab264c4a Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 10:09:47 -0400 Subject: [PATCH 10/56] delete passes --- playground/BTree/Program.cs | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index fa8ac3e27f8..ea3fbe70ad0 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -144,19 +144,16 @@ static unsafe void Main(string[] args) Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); Console.WriteLine("Time for deletion = " + deleteTime + " ns"); } - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); - // // do a range query to check again - // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List<(byte[], Value)> tombstones); - // Console.WriteLine("list length = " + tombstones.Count); - // foreach (var item in tombstones) - // { - // Console.WriteLine(item.Item2.value); - // } - // Console.WriteLine("Delete check passed "); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); + + // do a range query to check again + tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); + Debug.Assert(tombstones.Count == 4); + Console.WriteLine("Delete check passed "); //tree.Trim((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out int overflows); From 1be22c6a7a54be6f20c7d0b6972da5f07b2c54fa Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 13:47:05 -0400 Subject: [PATCH 11/56] tying up the streams API --- libs/common/RespWriteUtils.cs | 17 ++ libs/server/API/IGarnetApi.cs | 8 + libs/server/BTreeIndex/BTree.cs | 24 +++ libs/server/Resp/Parser/RespCommand.cs | 19 +++ libs/server/Resp/RespServerSession.cs | 12 ++ libs/server/Resp/StreamCommands.cs | 76 +++++++++ libs/server/Servers/ServerOptions.cs | 30 ++++ libs/server/StoreWrapper.cs | 5 + libs/server/Stream/Stream.cs | 124 ++++++++------- libs/server/Stream/StreamManager.cs | 7 +- .../cs/src/core/TsavoriteLog/TsavoriteLog.cs | 39 +++++ .../CommandInfoUpdater/SupportedCommand.cs | 1 + test/Garnet.test/BTreeTests.cs | 150 ++++++++++++++++++ 13 files changed, 446 insertions(+), 66 deletions(-) create mode 100644 libs/server/Resp/StreamCommands.cs create mode 100644 test/Garnet.test/BTreeTests.cs diff --git a/libs/common/RespWriteUtils.cs b/libs/common/RespWriteUtils.cs index f7a5fd9e7f5..b9083906e89 100644 --- a/libs/common/RespWriteUtils.cs +++ b/libs/common/RespWriteUtils.cs @@ -186,6 +186,23 @@ public static bool TryWriteSimpleString(ReadOnlySpan simpleString, ref byt return true; } + /// + /// Write a long as a simple string + /// + // public static bool TryWriteLongAsSimpleString(long value, ref byte* curr, byte* end) + // { + // // Simple strings are of the form "+cc\r\n" + // var longLength = NumUtils.CountDigits(value); + // var totalLen = 1 + longLength + 2; + // if (totalLen > (int)(end - curr)) + // return false; + + // *curr++ = (byte)'+'; + // NumUtils.LongToBytes(value, longLength, ref curr); + // WriteNewline(ref curr); + // return true; + // } + /// /// Write simple string /// diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 903619fde13..9a465fa9d68 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -554,6 +554,10 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi #endregion + #region Stream Methods + // GarnetStatus StreamAdd(ArgSlice key, ref ObjectInput input); + #endregion + #region Set Methods /// @@ -1408,6 +1412,10 @@ public interface IGarnetReadApi #endregion + #region Stream Methods + // GarnetStatus StreamLength(ArgSlice key, out long count); + #endregion + #region Geospatial Methods /// diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 5d05272f2a6..906f757b733 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -81,6 +81,30 @@ private void Free(ref BTreeNode* node) // } } + public static void FreeNode(ref BTreeNode* node) + { + if (node == null) + return; + + // If this is an internal node, free all its children first + if (node->info->type == BTreeNodeType.Internal) + { + for (int i = 0; i <= node->info->count; i++) + { + var child = node->data.children[i]; + FreeNode(ref child); + node->data.children[i] = null; + } + } + + // Free the memory handle + if (node->memoryHandle != null) + { + Marshal.FreeHGlobal((IntPtr)node->memoryHandle); + node = null; + } + } + /// /// Deallocates the memory allocated for the B+Tree /// diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 49e7e70c526..6dddeda608b 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -184,6 +184,10 @@ public enum RespCommand : ushort SREM, SUNIONSTORE, UNLINK, + XADD, + XLEN, + XRANGE, + XDEL, ZADD, ZDIFFSTORE, ZINCRBY, @@ -939,6 +943,21 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan } break; + case 'X': + if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nXADD\r\n"u8)) + { + return RespCommand.XADD; + } + else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nXLEN\r\n"u8)) + { + return RespCommand.XLEN; + } + else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nXDEL\r\n"u8)) + { + return RespCommand.XDEL; + } + break; + case 'Z': if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nZADD\r\n"u8)) { diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index cb9cec8afb9..3ac69aa1042 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -191,6 +191,11 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase // Threshold for slow log in ticks (0 means disabled) readonly long slowLogThreshold; + /// + /// Stream cache for the session + /// + internal readonly SessionStreamCache sessionStreamCache; + public RespServerSession( long id, INetworkSender networkSender, @@ -249,6 +254,10 @@ public RespServerSession( if (this.networkSender.GetMaxSizeSettings?.MaxOutputSize < sizeof(int)) this.networkSender.GetMaxSizeSettings.MaxOutputSize = sizeof(int); } + + // grab stream manager from storeWrapper + this.streamManager = storeWrapper.streamManager; + sessionStreamCache = new SessionStreamCache(); } internal void SetUserHandle(UserHandle userHandle) @@ -806,6 +815,9 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st RespCommand.SUNIONSTORE => SetUnionStore(ref storageApi), RespCommand.SDIFF => SetDiff(ref storageApi), RespCommand.SDIFFSTORE => SetDiffStore(ref storageApi), + // Stream Commands + RespCommand.XADD => StreamAdd(), + // RespCommand.XLEN => StreamLength(), _ => ProcessOtherCommands(cmd, ref storageApi) }; return success; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs new file mode 100644 index 00000000000..7899e3143b7 --- /dev/null +++ b/libs/server/Resp/StreamCommands.cs @@ -0,0 +1,76 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using Garnet.common; +using Tsavorite.core; + +namespace Garnet.server +{ + internal sealed unsafe partial class RespServerSession : ServerSessionBase + { + readonly StreamManager streamManager; + /// + /// STREAMADD + /// + /// + /// + private unsafe bool StreamAdd() + { + // Parse the stream key. + var key = parseState.GetArgSliceByRef(0); + + // Parse the id. We parse as string for easy pattern matching. + var idGiven = parseState.GetArgSliceByRef(1); + + // get the number of the remaining key-value pairs + var numPairs = parseState.Count - 2; + + // grab the rest of the input that will mainly be k-v pairs as entry to the stream. + byte* vPtr = parseState.GetArgSliceByRef(2).ptr - sizeof(int); + //int vsize = (int)(recvBufferPtr + bytesRead - vPtr); + int vsize = (int)(recvBufferPtr + endReadHead - vPtr); + SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + + + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) + { + cachedStream.AddEntry(vPtr, vsize, idGiven, numPairs, ref _output); + } + else + { + streamManager.StreamAdd(key, idGiven, vPtr, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream); + // since we added to a new stream that was not in the cache, try adding it to the cache + sessionStreamCache.TryAddStreamToCache(lastStreamKey, lastStream); + } + _ = ProcessOutputWithHeader(_output); + return true; + } + + /// + /// STREAMLENGTH + /// + /// + private bool StreamLength() + { + // parse the stream key. + var key = parseState.GetArgSliceByRef(0); + + ulong streamLength; + + // check if the stream exists in cache + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) + { + streamLength = cachedStream.Length(); + } + else + { + streamLength = streamManager.StreamLength(key); + } + // write back result + while (!RespWriteUtils.TryWriteInt64((long)streamLength, ref dcurr, dend)) + SendAndReset(); + return true; + } + } +} \ No newline at end of file diff --git a/libs/server/Servers/ServerOptions.cs b/libs/server/Servers/ServerOptions.cs index 297b257657e..a9cb966b9f4 100644 --- a/libs/server/Servers/ServerOptions.cs +++ b/libs/server/Servers/ServerOptions.cs @@ -99,6 +99,10 @@ public class ServerOptions /// public bool SkipRDBRestoreChecksumValidation = false; + public string StreamPageSize = "4m"; + + public string StreamMemorySize = "1g"; + /// /// Logger /// @@ -151,6 +155,32 @@ public long PubSubPageSizeBytes() return adjustedSize; } + /// + /// Get stream page size + /// + /// + public long StreamPageSizeBytes() + { + long size = ParseSize(StreamPageSize); + long adjustedSize = PreviousPowerOf2(size); + if (size != adjustedSize) + logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); + return adjustedSize; + } + + /// + /// Get stream memory size + /// + /// + public long StreamMemorySizeBytes() + { + long size = ParseSize(StreamMemorySize); + long adjustedSize = PreviousPowerOf2(size); + if (size != adjustedSize) + logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); + return adjustedSize; + } + /// /// Get segment size /// diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index d6438f48cee..6ece5e1f4ac 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -124,6 +124,8 @@ public sealed class StoreWrapper : IDisposable /// public readonly int databaseNum = 1; + internal readonly StreamManager streamManager; + /// /// Constructor /// @@ -212,6 +214,9 @@ public StoreWrapper( clusterProvider = clusterFactory.CreateClusterProvider(this); ctsCommit = new(); run_id = Generator.CreateHexId(); + + // initialize stream manager + this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes()); } /// diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index c844e36b3ba..cd1b28138fd 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -19,6 +19,11 @@ public enum XTRIMOpts NONE } + public enum XADDOpts + { + + } + public class StreamObject : IDisposable { readonly IDevice device; @@ -93,7 +98,6 @@ public void IncrementID(ref StreamID incrementedID) /// StreamID generated public void GenerateNextID(ref StreamID id) { - // ulong timestamp = (ulong)DateTimeOffset.UtcNow.ToUnixTimeMilliseconds(); ulong timestamp = (ulong)Stopwatch.GetTimestamp() / (ulong)(Stopwatch.Frequency / 1000); // if this is the first entry or timestamp is greater than last added entry @@ -118,16 +122,16 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) return true; } - // we have to parse user-defined ID - // this can be of following formats: - // 1. ts (seq = 0) - // 2. ts-* (auto-generate seq number) - // 3. ts-seq + // parse user-defined ID + // can be of following formats: + // 1. ts (seq = 0) + // 2. ts-* (auto-generate seq number) + // 3. ts-seq - // check if last character is a * + // last character is a * if (*(idSlice.ptr + idSlice.length - 1) == '*') { - // this has to be of format ts-*, so check if '-' is the preceding character + // has to be of format ts-*, check if '-' is the preceding character if (*(idSlice.ptr + idSlice.length - 2) != '-') { return false; @@ -168,7 +172,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) break; } } - // if '-' is not found, it has to be of format ts + // if '-' is not found, format should be just ts if (index == -1) { if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + idSlice.length)) @@ -241,56 +245,56 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int byte* tmpPtr = null; StreamID id = default; // take a lock to ensure thread safety - // _lock.WriteLock(); - // try - // { - // bool canParseID = parseIDString(idSlice, ref id); - // if (!canParseID) - // { - // // while (!RespWriteUtils.WriteError("ERR Syntax", ref curr, end)) - // // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // return; - // } - - // // add the entry to the log - // { - // bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); - // if (!enqueueInLog) - // { - // while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // return; - // } - - // var streamValue = new Value((ulong)retAddress); - - // bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); - // // bool added = true; - // if (!added) - // { - // while (!RespWriteUtils.WriteError("ERR StreamAdd failed", ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // return; - // } - // lastId.setMS(id.ms); - // lastId.setSeq(id.seq); - - // totalEntriesAdded++; - // // write back the ID of the entry added - // string idString = $"{id.getMS()}-{id.getSeq()}"; - // while (!RespWriteUtils.WriteSimpleString(idString, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // } - // } - // finally - // { - // // log.Commit(); - - // if (isMemory) ptrHandle.Dispose(); - // output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); - // _lock.WriteUnlock(); - - // } + _lock.WriteLock(); + try + { + bool canParseID = parseIDString(idSlice, ref id); + if (!canParseID) + { + while (!RespWriteUtils.TryWriteError("ERR Syntax", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + + // add the entry to the log + { + bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); + if (!enqueueInLog) + { + while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + + var streamValue = new Value((ulong)retAddress); + + bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); + // bool added = true; + if (!added) + { + while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + return; + } + lastId.setMS(id.ms); + lastId.setSeq(id.seq); + + totalEntriesAdded++; + // write back the ID of the entry added + string idString = $"{id.getMS()}-{id.getSeq()}"; + while (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + } + } + finally + { + // log.Commit(); + + if (isMemory) ptrHandle.Dispose(); + output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); + _lock.WriteUnlock(); + + } } @@ -373,8 +377,6 @@ unsafe bool parseCompleteID(ArgSlice idSlice, out StreamID streamID) return true; } - - /// public void Dispose() { diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index e89a369b107..7a1403b4a91 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -34,10 +34,9 @@ public StreamManager(long pageSize, long memorySize) /// /// key of last stream accessed (for cache) /// reference to last stream accessed (for cache) - /// Note: Can refactor some of this code to get rid of streamKey and lastStream params. public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream) { - // create a copy as we need to store this key in the dictionary + // copy key store this key in the dictionary byte[] key = new byte[keySlice.Length]; fixed (byte* keyPtr = key) Buffer.MemoryCopy(keySlice.ptr, keyPtr, keySlice.Length, keySlice.Length); @@ -69,7 +68,7 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, i _lock.WriteLock(); try { - // retry querying the dictionary to see if some other thread has created the stream + // retry to validate if some other thread has created the stream foundStream = streams.TryGetValue(key, out stream); if (!foundStream) { @@ -82,9 +81,7 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, i } else { - // we found the stream but it was not the one that we have in cache, so update the cache stream.AddEntry(value, valueLength, idSlice, numPairs, ref output); - // update last accessed stream key lastStream = stream; streamKey = key; } diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs index 9cacbed79a9..70490cdf775 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs @@ -844,6 +844,45 @@ public unsafe bool TryEnqueue(ReadOnlySpan entry, out long logicalAddress) return true; } + public unsafe bool TryEnqueueStreamEntry(byte* id, int idLength, int numPairs, byte* entry, int entryLength, out long logicalAddress) + { + logicalAddress = 0; + var length = idLength + sizeof(int) + entryLength; + int allocatedLength = headerSize + Align(length); + ValidateAllocatedLength(allocatedLength); + + epoch.Resume(); + + if (commitNum == long.MaxValue) throw new TsavoriteException("Attempting to enqueue into a completed log"); + + logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); + if (logicalAddress == 0) + { + epoch.Suspend(); + if (cannedException != null) throw cannedException; + return false; + } + + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); + // start writing + // first we copy the id + *(long*)(headerSize + physicalAddress) = *(long*)id; + *(long*)(headerSize + physicalAddress + 8) = *(long*)(id + sizeof(long)); + // Buffer.MemoryCopy(id, (void*)(headerSize + physicalAddress), idLength, idLength); + // then we copy the number of pairs + // Buffer.MemoryCopy(numPairsBytes, (void*)(headerSize + physicalAddress + idLength), numPairsBytesLength, numPairsBytesLength); + *(int*)(headerSize + physicalAddress + idLength) = numPairs; + // then we copy the entry + Buffer.MemoryCopy(entry, (void*)(headerSize + physicalAddress + idLength + sizeof(int)), entryLength, entryLength); + + SetHeader(length, (byte*)physicalAddress); + safeTailRefreshEntryEnqueued?.Signal(); + epoch.Suspend(); + if (AutoCommit) Commit(); + return true; + } + + /// /// Append a user-defined blittable struct header atomically to the log. /// diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index 9163630282a..cdea5ec619a 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -303,6 +303,7 @@ public class SupportedCommand new("WATCH", RespCommand.WATCH), new("WATCHMS", RespCommand.WATCHMS), new("WATCHOS", RespCommand.WATCHOS), + new("XADD", RespCommand.XADD), new("ZADD", RespCommand.ZADD), new("ZCARD", RespCommand.ZCARD), new("ZCOUNT", RespCommand.ZCOUNT), diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs new file mode 100644 index 00000000000..bbed293fe30 --- /dev/null +++ b/test/Garnet.test/BTreeTests.cs @@ -0,0 +1,150 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; +using Garnet.server; +using Garnet.server.BTreeIndex; +using NUnit.Framework; +using NUnit.Framework.Legacy; + +namespace Garnet.test +{ + using Node = BTreeNode; + using Value = Value; + + [TestFixture] + public unsafe class BTreeTests + { + static StreamID[] streamIDs; + static ulong N = 50000; + + [SetUp] + public void Setup() + { + streamIDs = new StreamID[N]; + for (ulong i = 0; i < N; i++) + { + streamIDs[i] = new StreamID(i + 1, 0); + } + } + + [TearDown] + public void TearDown() + { } + + [Test] + [Category("INIT")] + public void InitBTreeLeafNode() + { + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var leaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); + ClassicAssert.AreEqual(leaf->info->type, BTreeNodeType.Leaf); + ClassicAssert.AreEqual(leaf->info->count, 0); + + // free the leaf + BTree.FreeNode(ref leaf); + + leaf = null; + } + + [Test] + [Category("INIT")] + public void InitBTreeInternalNode() + { + var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var internalNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); + ClassicAssert.AreEqual(internalNode->info->type, BTreeNodeType.Internal); + ClassicAssert.AreEqual(internalNode->info->count, 0); + + // free the leaf + BTree.FreeNode(ref internalNode); + Marshal.FreeHGlobal((IntPtr)internalNode); + internalNode = null; + } + + [Test] + [Category("INSERT")] + public void Insert() + { + var tree = new BTree(4096); + ClassicAssert.AreEqual(tree.FastInserts, 0); + ClassicAssert.AreEqual(tree.LeafCount, 1); + ClassicAssert.AreEqual(tree.InternalCount, 0); + + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(i + 1)); + } + ClassicAssert.AreEqual(tree.FastInserts, N); + tree.Deallocate(); + } + + [Test] + [Category("LOOKUP")] + public void PointLookup() + { + var tree = new BTree(4096); + + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); + } + + for (ulong i = 0; i < N; i++) + { + ClassicAssert.AreEqual(tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0])).address, streamIDs[i].ms); + } + + tree.Deallocate(); + } + + [Test] + [Category("LOOKUP")] + public void RangeLookup() + { + var tree = new BTree(4096); + + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); + } + + int count = tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal, out Value endVal, out List list); + ClassicAssert.AreEqual(count, N - 1 - (N - 200) + 1); + ClassicAssert.AreEqual(list.Count, 0); + ClassicAssert.AreEqual(startVal.address, streamIDs[N - 200].ms); + ClassicAssert.AreEqual(endVal.address, streamIDs[N - 1].ms); + + tree.Deallocate(); + } + + [Test] + [Category("Delete")] + public void Delete() + { + var tree = new BTree(4096); + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); + } + + // delete 10% of keys at random + Random rand = new Random(); + uint delCount = 0; + for (ulong i = 0; i < N / 10; i++) + { + ulong idx = (ulong)rand.Next(0, (int)N); + bool deleted = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); + if (deleted) + { + delCount++; + } + } + ClassicAssert.AreEqual(tree.ValidCount, N - delCount); + tree.Deallocate(); + } + } +} From d8a41f45c13c92f5ab7e12cc0d6c428e13d44002 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 14:26:02 -0400 Subject: [PATCH 12/56] added command to RespCommandsInfo --- libs/resources/RespCommandsInfo.json | 60 +++++++++++++++++++ .../CommandInfoUpdater/SupportedCommand.cs | 1 + 2 files changed, 61 insertions(+) diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index c0a7b00a31f..c62f7807591 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -4828,6 +4828,66 @@ } ] }, + { + "Command": "XADD", + "Name": "XADD", + "IsInternal": false, + "Arity": -5, + "Flags": "DenyOom, Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Fast, Stream, Write", + "Tips": [ + "nondeterministic_output" + ], + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Notes": "UPDATE instead of INSERT because of the optional trimming feature", + "Flags": "RW, Update" + } + ], + "SubCommands": null + }, + { + "Command": "XLEN", + "Name": "XLEN", + "IsInternal": false, + "Arity": 2, + "Flags": "Fast, ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Fast, Read, Stream", + "Tips": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Notes": null, + "Flags": "RO" + } + ], + "SubCommands": null + }, { "Command": "ZADD", "Name": "ZADD", diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index cdea5ec619a..611b4ff0f6f 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -304,6 +304,7 @@ public class SupportedCommand new("WATCHMS", RespCommand.WATCHMS), new("WATCHOS", RespCommand.WATCHOS), new("XADD", RespCommand.XADD), + new("XLEN", RespCommand.XLEN), new("ZADD", RespCommand.ZADD), new("ZCARD", RespCommand.ZCARD), new("ZCOUNT", RespCommand.ZCOUNT), From 70f3bacc52af2de7e6bfbd8d6562fc2388884ca4 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 14:37:24 -0400 Subject: [PATCH 13/56] basic test with XADD and XLEN works --- libs/server/Resp/RespServerSession.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 3ac69aa1042..5a496cd1936 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -817,7 +817,7 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st RespCommand.SDIFFSTORE => SetDiffStore(ref storageApi), // Stream Commands RespCommand.XADD => StreamAdd(), - // RespCommand.XLEN => StreamLength(), + RespCommand.XLEN => StreamLength(), _ => ProcessOtherCommands(cmd, ref storageApi) }; return success; From 460d88fcf11a825fede9274fde2a1046a5d2c5b5 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 15:18:44 -0400 Subject: [PATCH 14/56] added XRANGE and XDEL - basic tests work --- libs/resources/RespCommandsInfo.json | 58 ++++++ libs/server/Resp/Parser/RespCommand.cs | 7 + libs/server/Resp/RespServerSession.cs | 2 + libs/server/Resp/StreamCommands.cs | 96 +++++++++ libs/server/StoreWrapper.cs | 3 +- libs/server/Stream/Stream.cs | 196 +++++++++++++++++- libs/server/Stream/StreamManager.cs | 6 +- .../CommandInfoUpdater/SupportedCommand.cs | 2 + 8 files changed, 364 insertions(+), 6 deletions(-) diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index c62f7807591..026bf1a5b5d 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -4859,6 +4859,35 @@ ], "SubCommands": null }, + { + "Command": "XDEL", + "Name": "XDEL", + "IsInternal": false, + "Arity": -3, + "Flags": "Fast, Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Fast, Stream, Write", + "Tips": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Notes": null, + "Flags": "RW, Delete" + } + ], + "SubCommands": null + }, { "Command": "XLEN", "Name": "XLEN", @@ -4888,6 +4917,35 @@ ], "SubCommands": null }, + { + "Command": "XRANGE", + "Name": "XRANGE", + "IsInternal": false, + "Arity": -4, + "Flags": "ReadOnly", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Read, Slow, Stream", + "Tips": null, + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Notes": null, + "Flags": "RO, Access" + } + ], + "SubCommands": null + }, { "Command": "ZADD", "Name": "ZADD", diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 6dddeda608b..c3174f540fc 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -1299,6 +1299,13 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan return RespCommand.UNLINK; } break; + + case 'X': + if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("XRANGE\r\n"u8)) + { + return RespCommand.XRANGE; + } + break; case 'Z': if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("ZCOUNT\r\n"u8)) diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 5a496cd1936..9d7e1e9f868 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -818,6 +818,8 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st // Stream Commands RespCommand.XADD => StreamAdd(), RespCommand.XLEN => StreamLength(), + RespCommand.XDEL => StreamDelete(), + RespCommand.XRANGE => StreamRange(), _ => ProcessOtherCommands(cmd, ref storageApi) }; return success; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 7899e3143b7..f723f7ad9ff 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -72,5 +72,101 @@ private bool StreamLength() SendAndReset(); return true; } + + /// + /// STREAMRANGE + /// + /// + /// + public unsafe bool StreamRange() + { + // command is of format: XRANGE key start end [COUNT count] + // we expect at least 3 arguments + if (parseState.Count < 3) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + } + + // parse the stream key + var key = parseState.GetArgSliceByRef(0); + + // parse start and end IDs + var startId = parseState.GetArgSliceByRef(1).ToString(); + var endId = parseState.GetArgSliceByRef(2).ToString(); + + int count = -1; + if (parseState.Count > 3) + { + // parse the count argument + var countStr = parseState.GetArgSliceByRef(4).ToString(); + if (!int.TryParse(countStr, out count)) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + } + } + + SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + + // check if the stream exists in cache + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) + { + cachedStream.ReadRange(startId, endId, count, ref _output); + } + else + { + streamManager.StreamRange(key, startId, endId, count, ref _output); + } + + + _ = ProcessOutputWithHeader(_output); + + return true; + } + + public bool StreamDelete() + { + // command is of format: XDEL key id [id ...] + // we expect at least 2 arguments + if (parseState.Count < 2) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + } + + // parse the stream key + var key = parseState.GetArgSliceByRef(0); + int deletedCount = 0; + + // for every id, parse and delete the stream entry + for (int i = 1; i < parseState.Count; i++) + { + // parse the id as string + var idGiven = parseState.GetArgSliceByRef(i); + + bool deleted; + // check if the stream exists in cache + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) + { + deleted = cachedStream.DeleteEntry(idGiven); + } + else + { + // delete the entry in the stream from the streamManager + deleted = streamManager.StreamDelete(key, idGiven, out StreamObject lastStream); + if (lastStream != null) + { + // since we deleted from a stream that was not in the cache, try adding it to the cache + sessionStreamCache.TryAddStreamToCache(key.ToArray(), lastStream); + } + } + + deletedCount = deleted ? deletedCount + 1 : deletedCount; + } + + // write back the number of entries deleted + while (!RespWriteUtils.TryWriteInt64(deletedCount, ref dcurr, dend)) + SendAndReset(); + return true; + } + } } \ No newline at end of file diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 6ece5e1f4ac..24f771b7fe7 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -216,7 +216,8 @@ public StoreWrapper( run_id = Generator.CreateHexId(); // initialize stream manager - this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes()); + // serverOptions.SubscriberRefreshFrequencyMs + this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0 ); } /// diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index cd1b28138fd..894525d8d71 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -9,6 +9,7 @@ using System.Diagnostics; using System.Buffers; using System.Runtime.CompilerServices; +using System.Buffers.Binary; namespace Garnet.server { @@ -39,10 +40,10 @@ public class StreamObject : IDisposable /// /// Directory where the log will be stored /// Page size of the log used for the stream - public StreamObject(string logDir, long pageSize, long memorySize) + public StreamObject(string logDir, long pageSize, long memorySize, int safeTailRefreshFreqMs) { device = logDir == null ? new NullDevice() : Devices.CreateLogDevice("streamLogs/" + logDir + "/streamLog", preallocateFile: false); - log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize }); + log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize, SafeTailRefreshFrequencyMs = safeTailRefreshFreqMs}); index = new BTree(device.SectorSize); totalEntriesAdded = 0; lastId = default; @@ -335,7 +336,7 @@ public unsafe bool DeleteEntry(ArgSlice idSlice) _lock.WriteLock(); try { - // deleted = index.Delete((byte*)Unsafe.AsPointer(ref entryID.idBytes[0])); + deleted = index.Delete((byte*)Unsafe.AsPointer(ref entryID.idBytes[0])); } finally { @@ -344,6 +345,195 @@ public unsafe bool DeleteEntry(ArgSlice idSlice) return deleted; } + public bool ParseCompleteStreamIDFromString(string idString, out StreamID id) + { + id = default; + string[] parts = idString.Split('-'); + if (parts.Length != 2) + { + return false; + } + if (!ulong.TryParse(parts[0], out ulong timestamp)) + { + return false; + } + if (!ulong.TryParse(parts[1], out ulong seq)) + { + return false; + } + + id.setMS(timestamp); + id.setSeq(seq); + return true; + } + + public bool ParseStreamIDFromString(string idString, out StreamID id) + { + id = default; + if (idString == "-" || idString == "+") + { + return false; + } + if (!idString.Contains('-')) + { + if (!ulong.TryParse(idString, out id.ms)) + { + return false; + } + id.setSeq(0); + return true; + } + return ParseCompleteStreamIDFromString(idString, out id); + } + + /// + /// Read entries from the stream from given range + /// + /// start of range + /// end of range + /// threshold to scanning + /// + public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndMemory output) + { + _lock.ReadLock(); + try + { + if (index.Count() == 0) + { + return; + } + + long startAddr, endAddr; + StreamID startID, endID; + if (min == "-") + { + byte[] idBytes = index.First().Key; + startID = new StreamID(idBytes); + } + else if (!ParseStreamIDFromString(min, out startID)) + { + return; + } + if (max == "+") + { + byte[] idBytes = index.Last().Key; + endID = new StreamID(idBytes); + } + else + { + if (!ParseStreamIDFromString(max, out endID)) + { + return; + } + //endID.seq = long.MaxValue; + endID.setSeq(long.MaxValue); + } + + int count = index.Get((byte*)Unsafe.AsPointer(ref startID.idBytes[0]), (byte*)Unsafe.AsPointer(ref endID.idBytes[0]), out Value startVal, out Value endVal, out var tombstones, limit); + startAddr = (long)startVal.address; + endAddr = (long)endVal.address + 1; + + byte* ptr = output.SpanByte.ToPointer(); + var curr = ptr; + var end = curr + output.Length; + MemoryHandle ptrHandle = default; + bool isMemory = false; + byte* tmpPtr = null; + int tmpSize = 0; + long readCount = 0; + + try + { + using (var iter = log.Scan(startAddr, endAddr, scanUncommitted: true)) + { + + // write length of how many entries we will print out + while (!RespWriteUtils.TryWriteArrayLength(count, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + + byte* e; + while (iter.GetNext(out var entry, out _, out long currentAddress, out long nextAddress)) + { + + var current = new Value((ulong)currentAddress); + // check if any tombstone t.address matches current + var tombstoneFound = false; + foreach (var tombstone in tombstones) + { + if (tombstone.address == current.address) + { + tombstoneFound = true; + break; + } + } + if (tombstoneFound) + { + continue; + } + + var entryBytes = entry.AsSpan(); + // check if the entry is actually one of the qualified keys + // parse ID for the entry which is the first 16 bytes + var idBytes = entryBytes.Slice(0, 16); + var ts = BinaryPrimitives.ReadInt64BigEndian(idBytes.Slice(0, sizeof(long))); + var seq = BinaryPrimitives.ReadInt64BigEndian(idBytes.Slice(8)); + string idString = $"{ts}-{seq}"; + Span numPairsBytes = entryBytes.Slice(16, 4); + int numPairs = BitConverter.ToInt32(numPairsBytes); + Span value = entryBytes.Slice(20); + + // we can already write back the ID that we read + while (!RespWriteUtils.TryWriteArrayLength(2, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + if (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) + { + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + } + + // print array length for the number of key-value pairs in the entry + while (!RespWriteUtils.TryWriteArrayLength(numPairs, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + + // write key-value pairs + fixed (byte* p = value) + { + e = p; + int read = 0; + read += (int)(e - p); + while (value.Length - read >= 4) + { + var orig = e; + if (!RespReadUtils.TryReadPtrWithLengthHeader(ref tmpPtr, ref tmpSize, ref e, e + entry.Length)) + { + return; + } + var o = new Span(tmpPtr, tmpSize).ToArray(); + while (!RespWriteUtils.TryWriteBulkString(o, ref curr, end)) + ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + read += (int)(e - orig); + } + } + readCount++; + if (limit != -1 && readCount == limit) + { + break; + } + } + } + } + finally + { + if (isMemory) ptrHandle.Dispose(); + output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); + } + } + finally + { + _lock.ReadUnlock(); + } + } + + unsafe bool parseCompleteID(ArgSlice idSlice, out StreamID streamID) { streamID = default; diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 7a1403b4a91..514005cadac 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -13,14 +13,16 @@ public sealed class StreamManager : IDisposable private Dictionary streams; long defPageSize; long defMemorySize; + int safeTailRefreshFreqMs; SingleWriterMultiReaderLock _lock = new SingleWriterMultiReaderLock(); - public StreamManager(long pageSize, long memorySize) + public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) { streams = new Dictionary(new ByteArrayComparer()); defPageSize = pageSize; defMemorySize = memorySize; + this.safeTailRefreshFreqMs = safeTailRefreshFreqMs; } /// @@ -73,7 +75,7 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, i if (!foundStream) { // stream was not found with this key so create a new one - StreamObject newStream = new StreamObject(null, defPageSize, defMemorySize); + StreamObject newStream = new StreamObject(null, defPageSize, defMemorySize, safeTailRefreshFreqMs); newStream.AddEntry(value, valueLength, idSlice, numPairs, ref output); streams.TryAdd(key, newStream); streamKey = key; diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index 611b4ff0f6f..64c64a1b12e 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -304,7 +304,9 @@ public class SupportedCommand new("WATCHMS", RespCommand.WATCHMS), new("WATCHOS", RespCommand.WATCHOS), new("XADD", RespCommand.XADD), + new("XDEL", RespCommand.XDEL), new("XLEN", RespCommand.XLEN), + new("XRANGE", RespCommand.XRANGE), new("ZADD", RespCommand.ZADD), new("ZCARD", RespCommand.ZCARD), new("ZCOUNT", RespCommand.ZCOUNT), From 579bd6e4ea88d5706e247c818a38c12e09fe9605 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 17:25:49 -0400 Subject: [PATCH 15/56] fixed bug with stream id parsing/encode/decode --- libs/server/Stream/Stream.cs | 98 +++++++++++++++++----------------- libs/server/Stream/StreamID.cs | 4 +- 2 files changed, 50 insertions(+), 52 deletions(-) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 894525d8d71..3beac084688 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -56,54 +56,44 @@ public StreamObject(string logDir, long pageSize, long memorySize, int safeTailR /// carries the incremented stream id public void IncrementID(ref StreamID incrementedID) { - while (true) - { - var originalMs = lastId.ms; - var originalSeq = lastId.seq; - - if (originalMs == long.MaxValue) - { - incrementedID = default; - return; - } + var originalMs = lastId.getMS(); + var originalSeq = lastId.getSeq(); - var newMs = originalMs; - var newSeq = originalSeq + 1; - - // if seq overflows, increment timestamp and reset seq - if (newSeq == 0) - { - newMs = originalMs + 1; - newSeq = 0; - } + if (originalMs == long.MaxValue) + { + incrementedID = default; + return; + } - // Use Interlocked.CompareExchange to ensure atomic update - var updatedMs = Interlocked.CompareExchange(ref lastId.ms, newMs, originalMs); - if (updatedMs == originalMs) - { - // Successfully updated ms, now update seq - Interlocked.Exchange(ref lastId.seq, newSeq); - incrementedID.setMS(newMs); - incrementedID.setSeq(newSeq); - return; - } + var newMs = originalMs; + var newSeq = originalSeq + 1; - // If we reach here, it means another thread has updated lastId.ms - // Retry the operation + // if seq overflows, increment timestamp and reset seq + if (newSeq == 0) + { + newMs += 1; + newSeq = 0; } + + incrementedID.setMS(newMs); + incrementedID.setSeq(newSeq); + } /// /// Generate the next stream ID /// /// StreamID generated - public void GenerateNextID(ref StreamID id) + public unsafe void GenerateNextID(ref StreamID id) { ulong timestamp = (ulong)Stopwatch.GetTimestamp() / (ulong)(Stopwatch.Frequency / 1000); + // read existing timestamp in big endian format + var lastTs = lastId.getMS(); // if this is the first entry or timestamp is greater than last added entry - if (totalEntriesAdded == 0 || timestamp > lastId.ms) + if (totalEntriesAdded == 0 || timestamp > lastTs) { + // this will write timestamp in big endian format id.setMS(timestamp); id.setSeq(0); return; @@ -123,6 +113,8 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) return true; } + var lastIdDecodedTs = lastId.getMS(); + // parse user-defined ID // can be of following formats: // 1. ts (seq = 0) @@ -145,13 +137,13 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) { return false; } - - // check if timestamp is greater than last added entry - if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + + // check if timestamp is greater than last added entry's decoded ts + if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { return false; } - else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { IncrementID(ref id); } @@ -181,11 +173,11 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) return false; } // check if timestamp is greater than last added entry - if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { return false; } - else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { IncrementID(ref id); } @@ -212,17 +204,19 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) return false; } - if (totalEntriesAdded != 0 && timestamp < lastId.getMS()) + if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { return false; } - else if (totalEntriesAdded != 0 && timestamp == lastId.getMS()) + else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { - if (seq <= lastId.getSeq()) + if (seq <= lastId.seq) { return false; } } + // use ID and seq given by user + // encode while storing id.setMS(timestamp); id.setSeq(seq); } @@ -277,11 +271,12 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); return; } - lastId.setMS(id.ms); - lastId.setSeq(id.seq); + // copy encoded ms and seq + lastId.ms = (id.ms); + lastId.seq = (id.seq); totalEntriesAdded++; - // write back the ID of the entry added + // write back the decoded ID of the entry added string idString = $"{id.getMS()}-{id.getSeq()}"; while (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); @@ -376,11 +371,13 @@ public bool ParseStreamIDFromString(string idString, out StreamID id) } if (!idString.Contains('-')) { - if (!ulong.TryParse(idString, out id.ms)) + + if (!ulong.TryParse(idString, out ulong ms)) { return false; } - id.setSeq(0); + id.setMS(ms); + id.setSeq(0); return true; } return ParseCompleteStreamIDFromString(idString, out id); @@ -475,8 +472,10 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM // check if the entry is actually one of the qualified keys // parse ID for the entry which is the first 16 bytes var idBytes = entryBytes.Slice(0, 16); - var ts = BinaryPrimitives.ReadInt64BigEndian(idBytes.Slice(0, sizeof(long))); - var seq = BinaryPrimitives.ReadInt64BigEndian(idBytes.Slice(8)); + var ts = BinaryPrimitives.ReadUInt64BigEndian(idBytes.Slice(0, 8)); + var seq = BinaryPrimitives.ReadUInt64BigEndian(idBytes.Slice(8, 8)); + // var ts = BitConverter.ToUInt64(idBytes.Slice(0, 8)); + // var seq = BitConverter.ToUInt64(idBytes.Slice(8, 8)); string idString = $"{ts}-{seq}"; Span numPairsBytes = entryBytes.Slice(16, 4); int numPairs = BitConverter.ToInt32(numPairsBytes); @@ -561,7 +560,6 @@ unsafe bool parseCompleteID(ArgSlice idSlice, out StreamID streamID) { return false; } - streamID.setMS(timestamp); streamID.setSeq(seq); return true; diff --git a/libs/server/Stream/StreamID.cs b/libs/server/Stream/StreamID.cs index 05150c02857..1cf32dfe649 100644 --- a/libs/server/Stream/StreamID.cs +++ b/libs/server/Stream/StreamID.cs @@ -38,12 +38,12 @@ public void setSeq(ulong seq) public ulong getMS() { - return ms; + return BinaryPrimitives.ReadUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.ms), 8)); } public ulong getSeq() { - return seq; + return BinaryPrimitives.ReadUInt64BigEndian(new Span(Unsafe.AsPointer(ref this.seq), 8)); } public unsafe StreamID(byte[] inputBytes) From 40af237c8c9e5afcd96666b627a1f42826150ea4 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 18:37:40 -0400 Subject: [PATCH 16/56] added ACL Tests --- libs/server/Resp/StreamCommands.cs | 18 +++++- libs/server/Stream/StreamManager.cs | 6 +- test/Garnet.test/Resp/ACL/RespCommandTests.cs | 61 +++++++++++++++++++ 3 files changed, 80 insertions(+), 5 deletions(-) diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index f723f7ad9ff..5c94942d25f 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -107,18 +107,30 @@ public unsafe bool StreamRange() SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + bool success = false; + // check if the stream exists in cache if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { cachedStream.ReadRange(startId, endId, count, ref _output); + success = true; } else { - streamManager.StreamRange(key, startId, endId, count, ref _output); + success = streamManager.StreamRange(key, startId, endId, count, ref _output); + } + if (success) + { + _ = ProcessOutputWithHeader(_output); + } + else{ + //return empty array + while (!RespWriteUtils.TryWriteArrayLength(0, ref dcurr, dend)) + SendAndReset(); + return true; } - - _ = ProcessOutputWithHeader(_output); + // _ = ProcessOutputWithHeader(_output); return true; } diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 514005cadac..8af7b59eca5 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -127,7 +127,7 @@ public unsafe ulong StreamLength(ArgSlice keySlice) /// end of range /// threshold to limit scanning /// - public void StreamRange(ArgSlice keySlice, string start, string end, int count, ref SpanByteAndMemory output) + public unsafe bool StreamRange(ArgSlice keySlice, string start, string end, int count, ref SpanByteAndMemory output) { var key = keySlice.ToArray(); if (streams != null && streams.Count > 0) @@ -135,9 +135,11 @@ public void StreamRange(ArgSlice keySlice, string start, string end, int count, bool foundStream = streams.TryGetValue(key, out StreamObject stream); if (foundStream) { - // stream.ReadRange(start, end, count, ref output); + stream.ReadRange(start, end, count, ref output); + return true; } } + return false; } /// diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index 1a2e8924b3d..1d431154762 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -6346,6 +6346,67 @@ static async Task DoGeoSearchStoreAsync(GarnetClient client) } } + [Test] + public async Task XADDACLsAsync() + { + int count = 0; + await CheckCommandsAsync( + "XADD", + [DoXAddAsync] + ); + + async Task DoXAddAsync(GarnetClient client) + { + string val = await client.ExecuteForStringResultAsync("XADD", ["foo", "*", $"bar--{count}", "fizz"]); + ClassicAssert.IsNotNull(val); + } + } + + [Test] + public async Task XLENACLsAsync() + { + await CheckCommandsAsync( + "XLEN", + [DoXLenAsync] + ); + + async Task DoXLenAsync(GarnetClient client) + { + long val = await client.ExecuteForLongResultAsync("XLEN", ["foo"]); + ClassicAssert.AreEqual(0, val); + } + } + + [Test] + public async Task XRangeACLsAsync() + { + await CheckCommandsAsync( + "XRANGE", + [DoXRangeAsync] + ); + + async Task DoXRangeAsync(GarnetClient client) + { + var val = await client.ExecuteForStringArrayResultAsync("XRANGE", ["foo", "-", "+"]); + ClassicAssert.AreEqual(0, val.Length); + } + } + + [Test] + public async Task XDELACLsAsync() + { + await CheckCommandsAsync( + "XDEL", + [DoXDelAsync] + ); + + async Task DoXDelAsync(GarnetClient client) + { + long val = await client.ExecuteForLongResultAsync("XDEL", ["foo", "1"]); + ClassicAssert.AreEqual(0, val); + } + } + [Test] public async Task ZAddACLsAsync() { From 83acdc7fc88767982c1e0b6f8040a20961823747 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 20:06:15 -0400 Subject: [PATCH 17/56] added RespStreamTests --- test/Garnet.test/BTreeTests.cs | 1 - test/Garnet.test/RespStreamTests.cs | 210 ++++++++++++++++++++++++++++ 2 files changed, 210 insertions(+), 1 deletion(-) create mode 100644 test/Garnet.test/RespStreamTests.cs diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index bbed293fe30..2b2217e8f7d 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -12,7 +12,6 @@ namespace Garnet.test { - using Node = BTreeNode; using Value = Value; [TestFixture] diff --git a/test/Garnet.test/RespStreamTests.cs b/test/Garnet.test/RespStreamTests.cs new file mode 100644 index 00000000000..78c636e7a9e --- /dev/null +++ b/test/Garnet.test/RespStreamTests.cs @@ -0,0 +1,210 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Globalization; +using System.Linq; +using System.Text; +using System.Threading.Tasks; +using Embedded.server; +using Garnet.common; +using Garnet.server; +using NUnit.Framework; +using NUnit.Framework.Legacy; +using StackExchange.Redis; +using Tsavorite.core; + +namespace Garnet.test +{ + [TestFixture] + public class RespStreamTests + { + protected GarnetServer server; + const string chars = "ABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789"; + Random random; + static ulong N = 5; + + [SetUp] + public void Setup() + { + TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true); + server.Start(); + random = new Random(); + + // write to one stream to test for range scans + var streamKey = "rangeScan"; + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + for (ulong i = 0; i < N; i++) + { + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue); + } + } + + [TearDown] + public void TearDown() + { + server.Dispose(); + TestUtils.DeleteDirectory(TestUtils.MethodTestDir); + } + + public string GenerateRandomString(int length) + { + return new string(Enumerable.Repeat(chars, length) + .Select(s => s[random.Next(s.Length)]).ToArray()); + } + + #region STREAMIDTests + [Test] + public void StreamAddAutoGenIdTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "add"; + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue); + ClassicAssert.IsTrue(retId.ToString().Contains("-")); + } + + [Test] + public void StreamAddUserDefinedTsTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "addTs"; + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue, $"{1}"); + ClassicAssert.IsTrue(retId.ToString().Contains("-")); + } + + [Test] + public void StreamAddUserDefinedIdTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "addId"; + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue, $"{1}-0"); + ClassicAssert.IsTrue(retId.ToString().Contains("-")); + } + #endregion + + #region STREAMOperationsTests + [Test] + public void StreamAddAndLengthTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "length"; + var count = 0; + for (ulong i = 0; i < N; i++) + { + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue); + count++; + } + ClassicAssert.AreEqual(count, N); + + var length = db.StreamLength(streamKey); + ClassicAssert.AreEqual(length, N); + } + + [Test] + public void StreamRangeExistingTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + var streamKey = "rangeScan"; + var range = db.StreamRange(streamKey, "-", "+"); + ClassicAssert.AreEqual(range.Length, N); + } + + [Test] + public void StreamRangeNonExistingTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + var streamKey = "nonExistingRangeScan"; + var range = db.StreamRange(streamKey, "-", "+"); + ClassicAssert.AreEqual(range.Length, 0); + } + + [Test] + public void StreamRangeWithCountTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + var streamKey = "rangeScan"; + int limit = 2; + var range = db.StreamRange(streamKey, "-", "+", limit); + ClassicAssert.AreEqual(range.Length, limit); + } + + [Test] + public void StreamDeleteSingleTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "delOne"; + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue, $"{1}-0"); + + var delCount = db.StreamDelete(streamKey, [retId]); + ClassicAssert.AreEqual(delCount, 1); + } + + [Test] + [Category("Delete")] + public void StreamDeleteMultipleTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "delMultiple"; + var count = 0; + for (ulong i = 0; i < N; i++) + { + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue, $"{i + 1}-0"); + count++; + } + ClassicAssert.AreEqual(count, N); + + // Pick arbitrary 2 unique indices between 0 and N and store each index in a set + int numToDelete = 2; + var indices = new HashSet(); + while (indices.Count < numToDelete) + { + indices.Add(random.Next(0, (int)N)); + } + + var eIds = new RedisValue[numToDelete]; + int c = 0; + foreach (var idx in indices) + { + eIds[c++] = $"{idx + 1}-0"; + } + + var delCount = db.StreamDelete(streamKey, eIds); + ClassicAssert.AreEqual(delCount, indices.Count); + } + + + #endregion + } +} \ No newline at end of file From 5444205f4205fb2993c97a54ebace54347a604c3 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 20:12:47 -0400 Subject: [PATCH 18/56] minor cleanup --- libs/server/API/IGarnetApi.cs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs index 9a465fa9d68..903619fde13 100644 --- a/libs/server/API/IGarnetApi.cs +++ b/libs/server/API/IGarnetApi.cs @@ -554,10 +554,6 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi #endregion - #region Stream Methods - // GarnetStatus StreamAdd(ArgSlice key, ref ObjectInput input); - #endregion - #region Set Methods /// @@ -1412,10 +1408,6 @@ public interface IGarnetReadApi #endregion - #region Stream Methods - // GarnetStatus StreamLength(ArgSlice key, out long count); - #endregion - #region Geospatial Methods /// From 5cab8b7295ca8814ec25e541432bea274121ba9f Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sat, 22 Mar 2025 20:33:39 -0400 Subject: [PATCH 19/56] added cleaner command strings for incorrect arguments --- libs/server/Resp/CmdStrings.cs | 9 ++++++++- libs/server/Resp/StreamCommands.cs | 13 +++++++++++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index b0ba2d0719f..61dca17e595 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -261,7 +261,14 @@ static partial class CmdStrings public static ReadOnlySpan RESP_ERR_FLUSHALL_READONLY_REPLICA => "ERR You can't write against a read only replica."u8; public static ReadOnlySpan RESP_ERR_DEUBG_DISALLOWED => @"ERR DEBUG command not allowed. If the EnableDebugCommand option is set to ""local"", you can run it from a local connection, otherwise you need to set this option in the configuration file, and then restart the server."u8; - + + // + /// Response strings for Stream commands + /// + public static ReadOnlySpan RESP_ERR_XADD_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xadd' command"u8; + public static ReadOnlySpan RESP_ERR_XLEN_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xlen' command"u8; + public static ReadOnlySpan RESP_ERR_XRANGE_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xrange' command"u8; + public static ReadOnlySpan RESP_ERR_XDEL_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xdel' command"u8; /// /// Response string templates diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 5c94942d25f..dc1101443bf 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -17,6 +17,11 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase /// private unsafe bool StreamAdd() { + if (parseState.Count < 4) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_XADD_WRONG_NUM_ARGS); + } + // Parse the stream key. var key = parseState.GetArgSliceByRef(0); @@ -53,6 +58,10 @@ private unsafe bool StreamAdd() /// private bool StreamLength() { + if (parseState.Count != 1) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_XLEN_WRONG_NUM_ARGS); + } // parse the stream key. var key = parseState.GetArgSliceByRef(0); @@ -84,7 +93,7 @@ public unsafe bool StreamRange() // we expect at least 3 arguments if (parseState.Count < 3) { - return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + return AbortWithErrorMessage(CmdStrings.RESP_ERR_XRANGE_WRONG_NUM_ARGS); } // parse the stream key @@ -141,7 +150,7 @@ public bool StreamDelete() // we expect at least 2 arguments if (parseState.Count < 2) { - return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + return AbortWithErrorMessage(CmdStrings.RESP_ERR_XDEL_WRONG_NUM_ARGS); } // parse the stream key From d3551d898f871833b46844b3efe78affc473b9d3 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 23 Mar 2025 10:36:57 -0400 Subject: [PATCH 20/56] cleanup; removed TRIM code --- libs/server/BTreeIndex/BTreeTrim.cs | 351 ---------------------------- libs/server/Stream/Stream.cs | 13 -- libs/server/Stream/StreamManager.cs | 27 --- playground/BTree/Program.cs | 4 - test/Garnet.test/BTreeTests.cs | 1 - 5 files changed, 396 deletions(-) delete mode 100644 libs/server/BTreeIndex/BTreeTrim.cs diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs deleted file mode 100644 index 287cb354328..00000000000 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ /dev/null @@ -1,351 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Diagnostics; -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; - -namespace Garnet.server.BTreeIndex -{ - public unsafe partial class BTree - { - public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) - { - underflowingNodes = 0; - BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; - BTreeNode* leaf = null; - validKeysRemoved = 0; - numLeavesDeleted = 0; - // first find the leaf node that could contain the key - TraverseToLeaf(ref leaf, ref nodesTraversed, key, out int[] internalSlots); - - // find the index for the key in the leaf node. Note: this would return the index of the first key greater than or equal to the given key - var index = leaf->LowerBound(key); - headValidValue = leaf->GetValue(index); - headValidKey = new byte[BTreeNode.KEY_SIZE]; - var headValidKeyPtr = leaf->GetKey(index); - Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); - - // TODO: shift entries in current leaf... for now, simply insert tombstones until this point - for (var i = 0; i < index; i++) - { - leaf->SetValueValid(i, false); - leaf->info->validCount--; - } - - if (leaf == head) - { - // we are already at head so we don't need to do any other traversals - numLeavesDeleted = 0; - return; - } - - // we will now traverse the leaf level of the tree and delete all preceding nodes - BTreeNode* node = leaf->info->previous; - - // # nodes to traverse in the subtree rooted at the leaf's parent (leaf is at nodesTraversed[0]). - // We subtract one since we delete preceding nodes. - var nodesToTraverseInSubtree = internalSlots[1] - 1; - uint deletedValidCount = 0; - - while (node != null) - { - var count = node->info->count; - var validCount = node->info->validCount; - if (nodesToTraverseInSubtree >= 0) - { - deletedValidCount += validCount; - nodesToTraverseInSubtree--; - } - - BTreeNode* prev = node->info->previous; - if (prev == null) - { - // should have reached the head, so do a sanity check - Debug.Assert(node == head); - } - - // update stats - stats.numLeafNodes--; - stats.numKeys -= count; - stats.numValidKeys -= validCount; - validKeysRemoved += validCount; - - // deallocate the node - node->Deallocate(); - Marshal.FreeHGlobal((IntPtr)node); - numLeavesDeleted++; - - // assign node to temp to continue - node = prev; - } - leaf->info->previous = null; - // set leaf as the new head - head = leaf; - - // now we will traverse the internal nodes (except root) and delete all preceding nodes - for (int i = 1; i < stats.depth - 1; i++) - { - // first handle the node in the nodesTraversed - node = nodesTraversed[i]; - var slotOfKey = internalSlots[i]; - - if (slotOfKey > 0) - { - // shift keys and children leftwards until slotOfKey (inclusive) - Buffer.MemoryCopy(node->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, node->keys, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(node->data.children + (slotOfKey - 1) + 1, node->data.children, ((slotOfKey - 1)) * sizeof(BTreeNode*), ((slotOfKey - 1)) * sizeof(BTreeNode*)); - } - - var prev_count = node->info->count; - // update count in node - node->info->count -= slotOfKey; - nodesTraversed[i]->info->validCount -= deletedValidCount; - - if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && node->info->count < BTreeNode.INTERNAL_CAPACITY / 2) - { - // TODO: handle underflow... for now, simply track how many such nodes we may have - underflowingNodes++; - } - - // reset deleted valid count for next level - deletedValidCount = 0; - - // next, handle all preceding internal nodes - node = nodesTraversed[i]->info->previous; - while (node != null) - { - BTreeNode* temp = node->info->previous; - if (nodesToTraverseInSubtree >= 0) - { - deletedValidCount += node->info->validCount; - nodesToTraverseInSubtree--; - } - - node->Deallocate(); - - Marshal.FreeHGlobal((IntPtr)node); - - // update stats - stats.numInternalNodes--; - node = temp; - } - // set the previous of nodesTraversed[i] to null as it is the new head in the level - nodesTraversed[i]->info->previous = null; - - // handle corner case where slotOfKey in the internal node points to the last child => after deletion, only one child remains. - // in this case, delete all parent levels and re-assign root. - if (i + 1 < stats.depth) - { - var nextSlot = internalSlots[i + 1]; - if (nextSlot == nodesTraversed[i + 1]->info->count) - { - BTreeNode* newRoot = nodesTraversed[i]; - var orig_depth = stats.depth; - for (int j = i + 1; j < orig_depth; j++) - { - BTreeNode* curr = nodesTraversed[j]; - while (curr != null) - { - BTreeNode* pre = curr->info->previous; - curr->Deallocate(); - Marshal.FreeHGlobal((IntPtr)curr); - stats.numInternalNodes--; - curr = pre; - } - stats.depth--; - } - root = newRoot; - break; - } - } - } - } - - public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) - { - int underflowingNodes; - TrimByID(key, out underflowingNodes, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); - } - - // function to trim the tree up to a given length - // for every internal node, we will scan the children from right to left and check its valid count from the info - // if the valid count is less than the length, we will keep the node and all its children to the right - // if the valid count is greater than or equal to the length, we add this node to the nodesTraversed at the current depth and continue from this child at the next level. - // once we reach the leaf level, we will trim everything to the left, and subsequently traverse the nodesTraversed and perform the same operation at every level. - public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nodesTraversed, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) - { - - var depth = stats.depth - 1; - ulong currentValidCount = 0; - BTreeNode* current = node; - int[] internalSlots = new int[MAX_TREE_DEPTH]; - int underflowingNodes = 0; - validKeysRemoved = 0; - numLeavesDeleted = 0; - headValidKey = new byte[BTreeNode.KEY_SIZE]; - // if the length is greater than the total number of valid keys, we will not trim anything - if (length >= stats.numValidKeys) - { - headValidValue = current->GetValue(0); - Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); - return; - } - while (depth > 0) - { - if (current->info->type == BTreeNodeType.Internal) - { - for (var i = current->info->count; i >= 0; i--) - { - // get the child node - BTreeNode* child = current->GetChild(i); - // if adding the child node's valid count wille exceed the length, we will continue on this child. Otherwise, we will keep this node and all its children to the right. - if (currentValidCount + child->info->validCount >= length) - { - nodesTraversed[depth] = current; - internalSlots[depth] = i; - current = child; - break; - } - else - { - currentValidCount += child->info->validCount; - } - } - } - depth--; - } - - // we have reached the leaf level. We will now trim everything to the left of the current node. - headValidValue = current->GetValue(0); - Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); - - BTreeNode* leaf = current->info->previous; - // might have to make sure that we are in a leaf node - Debug.Assert(leaf->info->type == BTreeNodeType.Leaf); - - uint deletedValidCount = 0; - var nodesToTraverseInSubtree = internalSlots[depth + 1] - 1; - while (leaf != null) - { - var count = leaf->info->count; - var validCount = leaf->info->validCount; - if (nodesToTraverseInSubtree >= 0) - { - deletedValidCount += validCount; - nodesToTraverseInSubtree--; - } - BTreeNode* prev = leaf->info->previous; - if (prev == null) - { - // should have reached the head, so do a sanity check - Debug.Assert(leaf == head); - } - - // update stats - stats.numLeafNodes--; - stats.numKeys -= count; - stats.numValidKeys -= validCount; - validKeysRemoved += validCount; - - // deallocate the node - leaf->Deallocate(); - Marshal.FreeHGlobal((IntPtr)leaf); - numLeavesDeleted++; - - // assign node to temp to continue - leaf = prev; - } - current->info->previous = null; - // set current as the new head - head = current; - - // now we will traverse the nodesTraversed and delete every node to its left, except the root node - for (int i = 1; i <= stats.depth - 1; i++) - { - var slotOfKey = internalSlots[i]; - BTreeNode* inner; - inner = nodesTraversed[i]; - - if (inner == null) - { - break; - } - - if (slotOfKey > 0) - { - // shift keys and children from slotOfKey to the beginning - Buffer.MemoryCopy(inner->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, inner->keys, (slotOfKey - 1) * BTreeNode.KEY_SIZE, (slotOfKey - 1) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(inner->data.children + slotOfKey, inner->data.children, (slotOfKey) * sizeof(BTreeNode*), (slotOfKey) * sizeof(BTreeNode*)); - } - - var prev_count = inner->info->count; - // update count in node - inner->info->count -= slotOfKey; - - nodesTraversed[i]->info->validCount -= deletedValidCount; - - if (prev_count > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info->count < BTreeNode.INTERNAL_CAPACITY / 2) - { - // TODO: handle underflow... for now, simply track how many such nodes we may have - underflowingNodes++; - } - - // grab all validCounts for nodes that we are deleting. - // subtract from parent's validCount for those we deleted - deletedValidCount = 0; - nodesToTraverseInSubtree = slotOfKey - 1; - inner = inner->info->previous; - while (inner != null && inner != root) - { - BTreeNode* temp = inner->info->previous; - if (nodesToTraverseInSubtree >= 0) - { - deletedValidCount += inner->info->validCount; - nodesToTraverseInSubtree--; - } - inner->Deallocate(); - Marshal.FreeHGlobal((IntPtr)inner); - stats.numInternalNodes--; - inner = temp; - } - nodesTraversed[i]->info->previous = null; - - // check the subsequent level in the tree - // if slotOfKey points to the last child, then all parent levels will be deleted - if (i + 1 < stats.depth) - { - var nextSlot = internalSlots[i + 1]; - if (nextSlot == nodesTraversed[i + 1]->info->count) - { - BTreeNode* newRoot = nodesTraversed[i]; - var orig_depth = stats.depth; - for (int j = i + 1; j < orig_depth; j++) - { - BTreeNode* curr = nodesTraversed[j]; - while (curr != null) - { - BTreeNode* pre = curr->info->previous; - curr->Deallocate(); - Marshal.FreeHGlobal((IntPtr)curr); - stats.numInternalNodes--; - curr = pre; - } - stats.depth--; - } - // now that we have deleted all parent nodes, set root to correct node - root = newRoot; - break; - } - } - } - } - - public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) - { - BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; - TrimByLength(ref root, length, ref nodesTraversed, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); - } - } -} \ No newline at end of file diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 3beac084688..018503d28ef 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -5,7 +5,6 @@ using Tsavorite.core; using Garnet.server.BTreeIndex; using Garnet.common; -using System.Threading; using System.Diagnostics; using System.Buffers; using System.Runtime.CompilerServices; @@ -13,18 +12,6 @@ namespace Garnet.server { - public enum XTRIMOpts - { - MAXLEN, - MINID, - NONE - } - - public enum XADDOpts - { - - } - public class StreamObject : IDisposable { readonly IDevice device; diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 8af7b59eca5..8dfdf49f51e 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -168,33 +168,6 @@ public bool StreamDelete(ArgSlice keySlice, ArgSlice idSlice, out StreamObject l return false; } - /// - /// Trim a stream - /// - /// key/name of stream - /// parameter to trim by - /// option to trim by (length or ID) - /// # valid keys removed - /// - public bool StreamTrim(ArgSlice keySlice, ArgSlice trimValue, XTRIMOpts optType, out ulong validKeysRemoved) - { - bool foundStream; - var key = keySlice.ToArray(); - StreamObject stream; - validKeysRemoved = 0; - if (streams != null) - { - foundStream = streams.TryGetValue(key, out stream); - - if (foundStream) - { - // return stream.Trim(trimValue, optType, out validKeysRemoved); - } - } - // we return true if the stream does not exist (as Resp expects a 0 result) - return true; - } - /// public void Dispose() { diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index ea3fbe70ad0..cbef85630e0 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -155,10 +155,6 @@ static unsafe void Main(string[] args) Debug.Assert(tombstones.Count == 4); Console.WriteLine("Delete check passed "); - //tree.Trim((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out int overflows); - - //tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[0].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[550].idBytes[0]), out Value startVal12, out Value endVal12, out List<(byte[], Value)> tombstones1); - // print all times collected in a csv format Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); tree.Deallocate(); diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 2b2217e8f7d..0f71ba1d472 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -60,7 +60,6 @@ public void InitBTreeInternalNode() // free the leaf BTree.FreeNode(ref internalNode); - Marshal.FreeHGlobal((IntPtr)internalNode); internalNode = null; } From dcd33b9ae64c46c75dceb98f1a976c3355904e63 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 23 Mar 2025 10:51:10 -0400 Subject: [PATCH 21/56] cleanup; added more documentation about node structure --- libs/server/BTreeIndex/BTree.cs | 4 ++++ libs/server/BTreeIndex/BTreeDelete.cs | 5 +--- libs/server/BTreeIndex/BTreeInsert.cs | 7 +++++- libs/server/BTreeIndex/BTreeInternals.cs | 30 +++++++++++++++++++++++- test/Garnet.test/BTreeTests.cs | 14 ----------- 5 files changed, 40 insertions(+), 20 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 906f757b733..b9355a58c90 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -81,6 +81,10 @@ private void Free(ref BTreeNode* node) // } } + /// + /// Frees the memory allocated for a node + /// + /// public static void FreeNode(ref BTreeNode* node) { if (node == null) diff --git a/libs/server/BTreeIndex/BTreeDelete.cs b/libs/server/BTreeIndex/BTreeDelete.cs index cd6b49f0b60..07097b04831 100644 --- a/libs/server/BTreeIndex/BTreeDelete.cs +++ b/libs/server/BTreeIndex/BTreeDelete.cs @@ -1,9 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. - -using System; - namespace Garnet.server.BTreeIndex { public unsafe partial class BTree @@ -12,7 +9,7 @@ public unsafe partial class BTree /// Delete a key from the B+tree /// /// key to delete - /// + /// true if key was tombstoned public bool Delete(byte* key) { BTreeNode* leaf = null; diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 68d982dfc6e..6673dca2a51 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -9,7 +9,12 @@ namespace Garnet.server.BTreeIndex { public unsafe partial class BTree { - + /// + /// Insert a key-value pair into the B+tree. Directly inserts into the tail leaf node. + /// + /// + /// + /// true if insertion is successful public bool Insert(byte* key, Value value) { BTreeNode* leaf = null; diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 0203352c83b..aa7d4219b5d 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -61,9 +61,37 @@ public unsafe struct NodeInfo public int count; public BTreeNode* next; public BTreeNode* previous; - public uint validCount; + public uint validCount; // valid keys (non-tombstone keys) in the node. } + /// + /// Represents a node in the B+tree + /// Memory layout: + /// +-----------------------------------+ + /// | BTreeNode (HEADER_SIZE bytes) | + /// | - NodeInfo* info | + /// | - NodeData data | + /// | - byte* keys | + /// | - IntPtr* memoryHandle | + /// +-----------------------------------+ + /// | NodeInfo (METADATA_SIZE bytes) | + /// | - BTreeNodeType type | + /// | - int count | + /// | - BTreeNode* next | + /// | - BTreeNode* previous | + /// | - uint validCount | + /// +-----------------------------------+ + /// | Keys array: capacity * KEY_SIZE | + /// +-----------------------------------+ + /// | Data array: either Value[] (leaf) | + /// | or BTreeNode*[] (internal) | + /// +-----------------------------------+ + /// Expects an allocated block of memory (of size BTreeNode.PAGE_SIZE) to be passed as handle + /// Stores handle for deallocation + /// BTreeNode struct also contained within the 4KB block to allow pointers to created nodes to be passed around + /// as well as allow for on-demand allocation/deallocation. + /// NOTE: currently reverted to MemoryMarshal for allocation of handles due to undefined behavior with SectorAlignedMemory. + /// public unsafe struct BTreeNode { public static int HEADER_SIZE = sizeof(BTreeNode); diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 0f71ba1d472..87d408be6c0 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -49,20 +49,6 @@ public void InitBTreeLeafNode() leaf = null; } - [Test] - [Category("INIT")] - public void InitBTreeInternalNode() - { - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); - var internalNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); - ClassicAssert.AreEqual(internalNode->info->type, BTreeNodeType.Internal); - ClassicAssert.AreEqual(internalNode->info->count, 0); - - // free the leaf - BTree.FreeNode(ref internalNode); - internalNode = null; - } - [Test] [Category("INSERT")] public void Insert() From 80033345ff84e7800a48727ac0cd4911e9325310 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 23 Mar 2025 10:54:47 -0400 Subject: [PATCH 22/56] more cleanup --- playground/BTree/Program.cs | 2 +- test/Garnet.test/BTreeTests.cs | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index cbef85630e0..7e4c82d198d 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -19,7 +19,7 @@ static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* } static unsafe void Main(string[] args) { - var tree = new BTree(4096); + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); ulong N = 400000; bool verbose = true; bool sanity = false; diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 87d408be6c0..3447191597c 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -53,7 +53,7 @@ public void InitBTreeLeafNode() [Category("INSERT")] public void Insert() { - var tree = new BTree(4096); + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); ClassicAssert.AreEqual(tree.FastInserts, 0); ClassicAssert.AreEqual(tree.LeafCount, 1); ClassicAssert.AreEqual(tree.InternalCount, 0); @@ -70,7 +70,7 @@ public void Insert() [Category("LOOKUP")] public void PointLookup() { - var tree = new BTree(4096); + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); for (ulong i = 0; i < N; i++) { @@ -109,7 +109,7 @@ public void RangeLookup() [Category("Delete")] public void Delete() { - var tree = new BTree(4096); + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); for (ulong i = 0; i < N; i++) { tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); From 5f92120b1d58e6ffff9fdd97037e9b5739565e1c Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 27 Mar 2025 10:46:01 +0100 Subject: [PATCH 23/56] reverted visibility change to ByteArrayComparer to instead use .Instance --- libs/server/Resp/ByteArrayComparer.cs | 2 +- libs/server/Stream/SessionStreamCache.cs | 2 +- libs/server/Stream/StreamManager.cs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/libs/server/Resp/ByteArrayComparer.cs b/libs/server/Resp/ByteArrayComparer.cs index 190044cbfda..ed58f013864 100644 --- a/libs/server/Resp/ByteArrayComparer.cs +++ b/libs/server/Resp/ByteArrayComparer.cs @@ -25,7 +25,7 @@ public sealed class ByteArrayComparer : IEqualityComparer public bool Equals(byte[] left, byte[] right) => new ReadOnlySpan(left).SequenceEqual(new ReadOnlySpan(right)); - public ByteArrayComparer() { } + private ByteArrayComparer() { } /// public unsafe int GetHashCode(byte[] key) diff --git a/libs/server/Stream/SessionStreamCache.cs b/libs/server/Stream/SessionStreamCache.cs index 94f18bb3c0c..1831cca5ddb 100644 --- a/libs/server/Stream/SessionStreamCache.cs +++ b/libs/server/Stream/SessionStreamCache.cs @@ -9,7 +9,7 @@ namespace Garnet.server internal struct SessionStreamCache { const int DefaultCacheSize = 16; - readonly Dictionary streamCache = new Dictionary(DefaultCacheSize, new ByteArrayComparer()); + readonly Dictionary streamCache = new Dictionary(DefaultCacheSize, ByteArrayComparer.Instance); readonly byte[][] streamKeysCache = new byte[DefaultCacheSize][]; int cachedStreamsCount = 0; int front = 0; diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 8dfdf49f51e..7d8574ff7a7 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -19,7 +19,7 @@ public sealed class StreamManager : IDisposable public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) { - streams = new Dictionary(new ByteArrayComparer()); + streams = new Dictionary(ByteArrayComparer.Instance); defPageSize = pageSize; defMemorySize = memorySize; this.safeTailRefreshFreqMs = safeTailRefreshFreqMs; From ede575dc1474ca660e9b36bd8e06bcb82e0e8dad Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 27 Mar 2025 14:23:24 +0100 Subject: [PATCH 24/56] removed most of Unsafe.AsPointer() instances from BTree code --- libs/server/BTreeIndex/BTree.cs | 6 +- libs/server/BTreeIndex/BTreeInsert.cs | 112 ++++++++++++++++++----- libs/server/BTreeIndex/BTreeInternals.cs | 5 +- libs/server/BTreeIndex/BTreeLookup.cs | 2 - libs/server/Stream/StreamID.cs | 7 +- playground/BTree/Program.cs | 3 +- 6 files changed, 102 insertions(+), 33 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index b9355a58c90..7273e60b1b4 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -167,8 +167,7 @@ public KeyValuePair First() { return default; } - byte[] keyBytes = new byte[BTreeNode.KEY_SIZE]; - Buffer.MemoryCopy(leaf->GetKey(0), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + byte[] keyBytes = new ReadOnlySpan(leaf->GetKey(0), BTreeNode.KEY_SIZE).ToArray(); return new KeyValuePair(keyBytes, leaf->GetValue(0)); } @@ -183,8 +182,7 @@ public KeyValuePair Last() { return default; } - byte[] keyBytes = new byte[BTreeNode.KEY_SIZE]; - Buffer.MemoryCopy(leaf->GetKey(leaf->info->count - 1), Unsafe.AsPointer(ref keyBytes[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + byte[] keyBytes = new ReadOnlySpan(leaf->GetKey(leaf->info->count - 1), BTreeNode.KEY_SIZE).ToArray(); return new KeyValuePair(keyBytes, leaf->GetValue(leaf->info->count - 1)); } diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 6673dca2a51..4a2077eb15d 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -23,7 +23,18 @@ public bool Insert(byte* key, Value value) stats.numKeys++; stats.numValidKeys++; leaf = tail; - return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value,true); + return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value, true); + } + + public bool Insert(byte* key,ReadOnlySpan keySpan, Value value) + { + BTreeNode* leaf = null; + stats.totalFastInserts++; + stats.totalInserts++; + stats.numKeys++; + stats.numValidKeys++; + leaf = tail; + return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value, true); } public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, bool appendToLeaf = false) { @@ -55,9 +66,11 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse if (leaf->info->count < BTreeNode.LEAF_CAPACITY) { - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index) * BTreeNode.KEY_SIZE, (leaf->info->count - index) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info->count - index) * sizeof(Value), (leaf->info->count - index) * sizeof(Value)); - + // move keys to the right of index + var sourceSpan = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, (leaf->info->count - index) * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index) * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + leaf->SetKey(index, key); leaf->SetValue(index, value); leaf->info->count++; @@ -97,23 +110,47 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, { // new key goes to the new leaf var newIndex = index - leaf->info->count; - Buffer.MemoryCopy(leaf->keys + leaf->info->count * BTreeNode.KEY_SIZE, newLeaf->keys, newIndex * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + + // move the keys from old node to the new node using ReadOnlySpan + var sourceSpan = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(newLeaf->keys, newIndex * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + // add key to new leaf newLeaf->SetKey(newIndex, key); - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->data.values + leaf->info->count, newLeaf->data.values, newIndex * sizeof(Value), newIndex * sizeof(Value)); + var existingLeafKeysSpan = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); + var newLeafKeysSpan = new Span(newLeaf->keys + (newIndex + 1) * BTreeNode.KEY_SIZE, (BTreeNode.LEAF_CAPACITY - index) * BTreeNode.KEY_SIZE); + existingLeafKeysSpan.CopyTo(newLeafKeysSpan); + + var existingLeafValuesSpan = new ReadOnlySpan(leaf->data.values + leaf->info->count, newIndex * sizeof(Value)); + var newLeafValuesSpan = new Span(newLeaf->data.values, newIndex * sizeof(Value)); + existingLeafValuesSpan.CopyTo(newLeafValuesSpan); newLeaf->SetValue(newIndex, value); - Buffer.MemoryCopy(leaf->data.values + index, newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value), (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + + var existingLeafValuesSpan2 = new ReadOnlySpan(leaf->data.values + index, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + var newLeafValuesSpan2 = new Span(newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); + existingLeafValuesSpan2.CopyTo(newLeafValuesSpan2); newLeaf->info->validCount++; } else { - Buffer.MemoryCopy(leaf->keys + (leaf->info->count - 1) * BTreeNode.KEY_SIZE, newLeaf->keys, newLeaf->info->count * BTreeNode.KEY_SIZE, newLeaf->info->count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(leaf->keys + index * BTreeNode.KEY_SIZE, leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE, (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); + var existingLeafKeysSpan = new ReadOnlySpan(leaf->keys + (leaf->info->count - 1) * BTreeNode.KEY_SIZE, newLeaf->info->count * BTreeNode.KEY_SIZE); + var newLeafKeysSpan = new Span(newLeaf->keys, newLeaf->info->count * BTreeNode.KEY_SIZE); + existingLeafKeysSpan.CopyTo(newLeafKeysSpan); + + var existingLeafKeysSpan2 = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); + var newLeafKeysSpan2 = new Span(leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); + existingLeafKeysSpan2.CopyTo(newLeafKeysSpan2); leaf->SetKey(index, key); - Buffer.MemoryCopy(leaf->data.values + leaf->info->count - 1, newLeaf->data.values, newLeaf->info->count * sizeof(Value), newLeaf->info->count * sizeof(Value)); - Buffer.MemoryCopy(leaf->data.values + index, leaf->data.values + index + 1, (leaf->info->count - index - 1) * sizeof(Value), (leaf->info->count - index - 1) * sizeof(Value)); + var existingLeafValuesSpan = new ReadOnlySpan(leaf->data.values + leaf->info->count - 1, newLeaf->info->count * sizeof(Value)); + var newLeafValuesSpan = new Span(newLeaf->data.values, newLeaf->info->count * sizeof(Value)); + existingLeafValuesSpan.CopyTo(newLeafValuesSpan); + + var existingLeafValuesSpan2 = new ReadOnlySpan(leaf->data.values + index, (leaf->info->count - index - 1) * sizeof(Value)); + var newLeafValuesSpan2 = new Span(leaf->data.values + index + 1, (leaf->info->count - index - 1) * sizeof(Value)); + existingLeafValuesSpan2.CopyTo(newLeafValuesSpan2); leaf->SetValue(index, value); leaf->info->validCount++; } @@ -173,7 +210,9 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, ref BTreeNode* child, ref BTreeNode*[] nodesTraversed, int index, uint newValidCount) { // move all keys to the right - Buffer.MemoryCopy(node->keys + index * BTreeNode.KEY_SIZE, node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info->count - index) * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); + var sourceSpan = new ReadOnlySpan(node->keys + index * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info->count - index) * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); // move all children starting from index+1 to the right using a for loop for (var j = node->info->count; j > index; j--) { @@ -220,28 +259,55 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r if (index > nodeToSplit->info->count) { // child goes to newNode - Buffer.MemoryCopy(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + var existingNodeKeysSpan = new ReadOnlySpan(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + var newNodeKeysSpan = new Span(newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); + existingNodeKeysSpan.CopyTo(newNodeKeysSpan); newNode->SetKey(index - nodeToSplit->info->count - 1, key); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*), (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + index, newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); + + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + 1 + nodeToSplit->info->count, (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); + var newNodeChildrenSpan = new Span(newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); + existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); + + var existingNodeChildrenSpan2 = new ReadOnlySpan(nodeToSplit->data.children + 1 + index, newNode->info->count * sizeof(BTreeNode*)); + var newNodeChildrenSpan2 = new Span(newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); + existingNodeChildrenSpan2.CopyTo(newNodeChildrenSpan2); newNode->SetChild(index - nodeToSplit->info->count, child); key = nodeToSplit->GetKey(nodeToSplit->info->count); } else if (index == nodeToSplit->info->count) { - Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->data.children + 1, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); + var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); + var newNodeChildrenSpan = new Span(newNode->data.children + 1, newNode->info->count * sizeof(BTreeNode*)); + existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); newNode->SetChild(0, child); } else { // child goes to old node - Buffer.MemoryCopy(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); - Buffer.MemoryCopy(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE, (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); + var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + var existingNodeKeysSpan = new ReadOnlySpan(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); + var newNodeKeysSpan = new Span(nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); + existingNodeKeysSpan.CopyTo(newNodeKeysSpan); nodeToSplit->SetKey(index, key); - Buffer.MemoryCopy(nodeToSplit->data.children + nodeToSplit->info->count, newNode->data.children, newNode->info->count * sizeof(BTreeNode*), newNode->info->count * sizeof(BTreeNode*)); - Buffer.MemoryCopy(nodeToSplit->data.children + index + 1, nodeToSplit->data.children + index + 2, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*), (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); + + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); + var newNodeChildrenSpan = new Span(newNode->data.children, newNode->info->count * sizeof(BTreeNode*)); + existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); + + var existingNodeChildrenSpan2 = new ReadOnlySpan(nodeToSplit->data.children + index + 1, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); + var newNodeChildrenSpan2 = new Span(nodeToSplit->data.children + index + 2, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); + existingNodeChildrenSpan2.CopyTo(newNodeChildrenSpan2); nodeToSplit->SetChild(index + 1, child); key = nodeToSplit->GetKey(nodeToSplit->info->count); } diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index aa7d4219b5d..df1c449a8e0 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -152,8 +152,9 @@ public unsafe struct BTreeNode public void SetKey(int index, byte* keyData) { - byte* keyAddress = keys + (index * KEY_SIZE); - Buffer.MemoryCopy(keyData, keyAddress, KEY_SIZE, KEY_SIZE); + var sourceSpan = new ReadOnlySpan(keyData, KEY_SIZE); + var destinationSpan = new Span(keys + (index * KEY_SIZE), KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); } public void SetChild(int index, BTreeNode* child) diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs index bfeb9290cd5..862af6bbd18 100644 --- a/libs/server/BTreeIndex/BTreeLookup.cs +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -105,8 +105,6 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out var value = leaf->GetValue(i); if (!value.Valid) { - byte[] key = new byte[BTreeNode.KEY_SIZE]; - Buffer.MemoryCopy(leaf->GetKey(i), Unsafe.AsPointer(ref key[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); tombstones.Add(leaf->GetValue(i)); } else diff --git a/libs/server/Stream/StreamID.cs b/libs/server/Stream/StreamID.cs index 1cf32dfe649..a2ab0a99fb9 100644 --- a/libs/server/Stream/StreamID.cs +++ b/libs/server/Stream/StreamID.cs @@ -53,7 +53,12 @@ public unsafe StreamID(byte[] inputBytes) throw new ArgumentException("idBytes must be 16 bytes"); } - Buffer.MemoryCopy((byte*)Unsafe.AsPointer(ref inputBytes[0]), (byte*)Unsafe.AsPointer(ref idBytes[0]), 16, 16); + fixed (byte* idBytesPtr = idBytes) + { + var sourceSpan = new ReadOnlySpan(inputBytes); + var destinationSpan = new Span(idBytesPtr, 16); + sourceSpan.CopyTo(destinationSpan); + } } } } \ No newline at end of file diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 7e4c82d198d..3cdf9fa6bd5 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -20,7 +20,7 @@ static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* static unsafe void Main(string[] args) { var tree = new BTree((uint)BTreeNode.PAGE_SIZE); - ulong N = 400000; + ulong N = 4000000; bool verbose = true; bool sanity = false; if (args.Length > 0) @@ -90,6 +90,7 @@ static unsafe void Main(string[] args) Console.WriteLine("Time for querying = " + query_time + " ns"); } sw.Reset(); + Console.WriteLine("All inserted keys found"); // forward range query double[] selectivities = [0.01, 0.05, 0.1]; From 6e4f239b4d636ad9a7f6fd1d19a5b6d5b016a2bd Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 27 Mar 2025 23:15:24 +0100 Subject: [PATCH 25/56] swapped out memory allocation for B-tree to use NativeMemory.AlignedAlloc instead of Marshal.AllocHGlobal --- libs/server/BTreeIndex/BTree.cs | 8 +++++--- libs/server/BTreeIndex/BTreeInsert.cs | 9 ++++++--- playground/BTree/Program.cs | 2 +- test/Garnet.test/BTreeTests.cs | 3 ++- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 7273e60b1b4..20485602a81 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -32,7 +32,7 @@ public BTree(uint sectorSize) { bufferPool = new SectorAlignedBufferPool(1, BTreeNode.PAGE_SIZE); // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates = 1; root = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); head = tail = root; @@ -68,7 +68,8 @@ private void Free(ref BTreeNode* node) // Free the memory handle if (node->memoryHandle != null) { - Marshal.FreeHGlobal((IntPtr)node->memoryHandle); + // Marshal.FreeHGlobal((IntPtr)node->memoryHandle); + NativeMemory.Free(node->memoryHandle); stats.numDeallocates++; node = null; } @@ -104,7 +105,8 @@ public static void FreeNode(ref BTreeNode* node) // Free the memory handle if (node->memoryHandle != null) { - Marshal.FreeHGlobal((IntPtr)node->memoryHandle); + // Marshal.FreeHGlobal((IntPtr)node->memoryHandle); + NativeMemory.Free(node->memoryHandle); node = null; } } diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 4a2077eb15d..60cf471820c 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -83,7 +83,8 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) { // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newLeaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); @@ -229,7 +230,8 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); stats.numInternalNodes++; @@ -319,7 +321,8 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newRoot = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 3cdf9fa6bd5..b2bb9916f14 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -20,7 +20,7 @@ static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* static unsafe void Main(string[] args) { var tree = new BTree((uint)BTreeNode.PAGE_SIZE); - ulong N = 4000000; + ulong N = 40000; bool verbose = true; bool sanity = false; if (args.Length > 0) diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 3447191597c..c41cd5d9754 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -38,7 +38,8 @@ public void TearDown() [Category("INIT")] public void InitBTreeLeafNode() { - var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); + var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); var leaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); ClassicAssert.AreEqual(leaf->info->type, BTreeNodeType.Leaf); ClassicAssert.AreEqual(leaf->info->count, 0); From ce156675b6c2b259afa892dd1af6d12059d52562 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 27 Mar 2025 23:26:55 +0100 Subject: [PATCH 26/56] cleanup; removed unused imports + commented out code. --- libs/server/BTreeIndex/BTree.cs | 15 --------------- libs/server/BTreeIndex/BTreeInsert.cs | 9 ++------- libs/server/BTreeIndex/BTreeInternals.cs | 8 +------- libs/server/BTreeIndex/BTreeLookup.cs | 2 -- playground/BTree/Program.cs | 7 +------ 5 files changed, 4 insertions(+), 37 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 20485602a81..404bf302bbf 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -3,9 +3,7 @@ using System; using System.Collections.Generic; -using System.Runtime.CompilerServices; using System.Runtime.InteropServices; -using Tsavorite.core; namespace Garnet.server.BTreeIndex { @@ -23,15 +21,11 @@ public unsafe partial class BTree BTreeNode*[] rootToTailLeaf; // array of nodes from root to tail leaf public BTreeStats stats; // statistics about the tree - SectorAlignedBufferPool bufferPool; - /// /// Initializes a new instance of the class. /// public BTree(uint sectorSize) { - bufferPool = new SectorAlignedBufferPool(1, BTreeNode.PAGE_SIZE); - // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates = 1; root = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); @@ -68,18 +62,10 @@ private void Free(ref BTreeNode* node) // Free the memory handle if (node->memoryHandle != null) { - // Marshal.FreeHGlobal((IntPtr)node->memoryHandle); NativeMemory.Free(node->memoryHandle); stats.numDeallocates++; node = null; } - - // if (node->memoryHandle != null) - // { - // node->memoryHandle.Return(); - // stats.numDeallocates++; - // node->memoryHandle = null; - // } } /// @@ -105,7 +91,6 @@ public static void FreeNode(ref BTreeNode* node) // Free the memory handle if (node->memoryHandle != null) { - // Marshal.FreeHGlobal((IntPtr)node->memoryHandle); NativeMemory.Free(node->memoryHandle); node = null; } diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 60cf471820c..239fb5f00ee 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System; -using System.Diagnostics; using System.Runtime.InteropServices; namespace Garnet.server.BTreeIndex @@ -82,8 +81,6 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, int index) { - // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newLeaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); @@ -214,6 +211,7 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r var sourceSpan = new ReadOnlySpan(node->keys + index * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); var destinationSpan = new Span(node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info->count - index) * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); + // move all children starting from index+1 to the right using a for loop for (var j = node->info->count; j > index; j--) { @@ -229,8 +227,6 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public BTreeNode* CreateInternalNode(ref BTreeNode* node, int splitPos) { - // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newNode = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); @@ -320,8 +316,6 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) { - // var memoryBlock = bufferPool.Get(BTreeNode.PAGE_SIZE); - // var memoryBlock = (IntPtr*)Marshal.AllocHGlobal(BTreeNode.PAGE_SIZE).ToPointer(); var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newRoot = BTreeNode.Create(BTreeNodeType.Internal, memoryBlock); @@ -329,6 +323,7 @@ public void CreateNewRoot(byte* key, BTreeNode* newlySplitNode) // Set the new root's key. newRoot->info->count = 1; newRoot->SetKey(0, key); + // Set children: left child is the old root; right child is the newly split node. newRoot->SetChild(0, root); newRoot->SetChild(1, newlySplitNode); diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index df1c449a8e0..d7b243beb3e 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -5,7 +5,6 @@ using System.Numerics; using System.Runtime.InteropServices; using System.Runtime.Intrinsics.X86; -using Tsavorite.core; namespace Garnet.server.BTreeIndex { @@ -104,8 +103,8 @@ public unsafe struct BTreeNode public NodeInfo* info; public NodeData data; public byte* keys; - // public SectorAlignedMemory memoryHandle; public IntPtr* memoryHandle; + public static BTreeNode* Create(BTreeNodeType type, IntPtr* handle) { // Place the node header at the beginning of the block. @@ -279,11 +278,6 @@ public static int Compare(byte* key1, byte* key2) return new Span(key1, KEY_SIZE).SequenceCompareTo(new Span(key2, KEY_SIZE)); } } - - public void Deallocate() - { - // memoryHandle.Dispose(); - } } /// diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs index 862af6bbd18..e5837a64327 100644 --- a/libs/server/BTreeIndex/BTreeLookup.cs +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -1,10 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; using System.Collections.Generic; using System.Diagnostics; -using System.Runtime.CompilerServices; namespace Garnet.server.BTreeIndex { diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index b2bb9916f14..217b6b650e3 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -5,7 +5,6 @@ using System.Runtime.CompilerServices; using Garnet.server; using Garnet.server.BTreeIndex; -using Tsavorite.core; class Program { /// @@ -13,14 +12,10 @@ class Program /// /// /// - static unsafe void something(SectorAlignedBufferPool pool, SectorAlignedMemory* ptr) - { - pool.Return(*ptr); - } static unsafe void Main(string[] args) { var tree = new BTree((uint)BTreeNode.PAGE_SIZE); - ulong N = 40000; + ulong N = 400000; bool verbose = true; bool sanity = false; if (args.Length > 0) From e66d1136e97070b6ac465459fa3bf78b2fd8c986 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 27 Mar 2025 23:34:45 +0100 Subject: [PATCH 27/56] bug fix - mismatch in allocates v/s deallocates for B-tree --- libs/server/BTreeIndex/BTree.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 404bf302bbf..474f3b1f30b 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -27,7 +27,6 @@ public unsafe partial class BTree public BTree(uint sectorSize) { var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); - stats.numAllocates = 1; root = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); head = tail = root; root->info->next = root->info->previous = null; @@ -37,6 +36,7 @@ public BTree(uint sectorSize) stats = new BTreeStats(); stats.depth = 1; stats.numLeafNodes = 1; + stats.numAllocates = 1; } /// From 70bed70cc126235d3fb868349a61beb3f1fd3b47 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 11 May 2025 19:51:49 -0400 Subject: [PATCH 28/56] updates fixing PR comments --- libs/common/RespWriteUtils.cs | 17 ---------- libs/server/Servers/GarnetServerOptions.cs | 36 ++++++++++++++++++++++ libs/server/Servers/ServerOptions.cs | 30 ------------------ libs/server/StoreWrapper.cs | 3 +- libs/server/Stream/SessionStreamCache.cs | 2 +- 5 files changed, 38 insertions(+), 50 deletions(-) diff --git a/libs/common/RespWriteUtils.cs b/libs/common/RespWriteUtils.cs index b9083906e89..f7a5fd9e7f5 100644 --- a/libs/common/RespWriteUtils.cs +++ b/libs/common/RespWriteUtils.cs @@ -186,23 +186,6 @@ public static bool TryWriteSimpleString(ReadOnlySpan simpleString, ref byt return true; } - /// - /// Write a long as a simple string - /// - // public static bool TryWriteLongAsSimpleString(long value, ref byte* curr, byte* end) - // { - // // Simple strings are of the form "+cc\r\n" - // var longLength = NumUtils.CountDigits(value); - // var totalLen = 1 + longLength + 2; - // if (totalLen > (int)(end - curr)) - // return false; - - // *curr++ = (byte)'+'; - // NumUtils.LongToBytes(value, longLength, ref curr); - // WriteNewline(ref curr); - // return true; - // } - /// /// Write simple string /// diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index fd6a0d8f79e..db53263de40 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -456,6 +456,16 @@ public class GarnetServerOptions : ServerOptions /// public StateMachineDriver StateMachineDriver; + /// + /// Page size for BTree index for STREAM + /// + public string StreamPageSize = "4m"; + + /// + /// Memory for STREAM + /// + public string StreamMemorySize = "1g"; + /// /// Constructor /// @@ -617,6 +627,32 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, return kvSettings; } + /// + /// Get stream page size + /// + /// + public long StreamPageSizeBytes() + { + long size = ParseSize(StreamPageSize); + long adjustedSize = PreviousPowerOf2(size); + if (size != adjustedSize) + logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); + return adjustedSize; + } + + /// + /// Get stream memory size + /// + /// + public long StreamMemorySizeBytes() + { + long size = ParseSize(StreamMemorySize); + long adjustedSize = PreviousPowerOf2(size); + if (size != adjustedSize) + logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); + return adjustedSize; + } + /// /// Get memory size /// diff --git a/libs/server/Servers/ServerOptions.cs b/libs/server/Servers/ServerOptions.cs index a9cb966b9f4..297b257657e 100644 --- a/libs/server/Servers/ServerOptions.cs +++ b/libs/server/Servers/ServerOptions.cs @@ -99,10 +99,6 @@ public class ServerOptions /// public bool SkipRDBRestoreChecksumValidation = false; - public string StreamPageSize = "4m"; - - public string StreamMemorySize = "1g"; - /// /// Logger /// @@ -155,32 +151,6 @@ public long PubSubPageSizeBytes() return adjustedSize; } - /// - /// Get stream page size - /// - /// - public long StreamPageSizeBytes() - { - long size = ParseSize(StreamPageSize); - long adjustedSize = PreviousPowerOf2(size); - if (size != adjustedSize) - logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); - return adjustedSize; - } - - /// - /// Get stream memory size - /// - /// - public long StreamMemorySizeBytes() - { - long size = ParseSize(StreamMemorySize); - long adjustedSize = PreviousPowerOf2(size); - if (size != adjustedSize) - logger?.LogInformation($"Warning: using lower stream page size than specified (power of 2)"); - return adjustedSize; - } - /// /// Get segment size /// diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 24f771b7fe7..aaf8af35c1b 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -216,8 +216,7 @@ public StoreWrapper( run_id = Generator.CreateHexId(); // initialize stream manager - // serverOptions.SubscriberRefreshFrequencyMs - this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0 ); + this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0); } /// diff --git a/libs/server/Stream/SessionStreamCache.cs b/libs/server/Stream/SessionStreamCache.cs index 1831cca5ddb..4f6ee3ba288 100644 --- a/libs/server/Stream/SessionStreamCache.cs +++ b/libs/server/Stream/SessionStreamCache.cs @@ -6,7 +6,7 @@ namespace Garnet.server { - internal struct SessionStreamCache + internal class SessionStreamCache { const int DefaultCacheSize = 16; readonly Dictionary streamCache = new Dictionary(DefaultCacheSize, ByteArrayComparer.Instance); From b3561649237b36b0999f617ff0f84d91cac55930 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 11 May 2025 20:03:33 -0400 Subject: [PATCH 29/56] reverted an earlier change in Build.props following PR comment + changes to fix build errors. --- Directory.Build.props | 2 +- libs/server/BTreeIndex/BTree.cs | 4 ++-- libs/server/Resp/CmdStrings.cs | 5 +---- libs/server/Resp/StreamCommands.cs | 14 ++++++-------- libs/server/Stream/Stream.cs | 17 ++++++++--------- playground/BTree/Program.cs | 7 +------ 6 files changed, 19 insertions(+), 30 deletions(-) diff --git a/Directory.Build.props b/Directory.Build.props index ec0305dd560..470cd3f5592 100644 --- a/Directory.Build.props +++ b/Directory.Build.props @@ -12,7 +12,7 @@ true true true - false + true diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 474f3b1f30b..8ee6ca78f3c 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -47,7 +47,7 @@ private void Free(ref BTreeNode* node) { if (node == null) return; - + // If this is an internal node, free all its children first if (node->info->type == BTreeNodeType.Internal) { @@ -76,7 +76,7 @@ public static void FreeNode(ref BTreeNode* node) { if (node == null) return; - + // If this is an internal node, free all its children first if (node->info->type == BTreeNodeType.Internal) { diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index 61dca17e595..60e963c7c32 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -261,10 +261,7 @@ static partial class CmdStrings public static ReadOnlySpan RESP_ERR_FLUSHALL_READONLY_REPLICA => "ERR You can't write against a read only replica."u8; public static ReadOnlySpan RESP_ERR_DEUBG_DISALLOWED => @"ERR DEBUG command not allowed. If the EnableDebugCommand option is set to ""local"", you can run it from a local connection, otherwise you need to set this option in the configuration file, and then restart the server."u8; - - // - /// Response strings for Stream commands - /// + public static ReadOnlySpan RESP_ERR_XADD_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xadd' command"u8; public static ReadOnlySpan RESP_ERR_XLEN_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xlen' command"u8; public static ReadOnlySpan RESP_ERR_XRANGE_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xrange' command"u8; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index dc1101443bf..7ae68bb4473 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -1,7 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System; using Garnet.common; using Tsavorite.core; @@ -13,7 +12,6 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase /// /// STREAMADD /// - /// /// private unsafe bool StreamAdd() { @@ -85,7 +83,6 @@ private bool StreamLength() /// /// STREAMRANGE /// - /// /// public unsafe bool StreamRange() { @@ -132,11 +129,12 @@ public unsafe bool StreamRange() { _ = ProcessOutputWithHeader(_output); } - else{ - //return empty array - while (!RespWriteUtils.TryWriteArrayLength(0, ref dcurr, dend)) - SendAndReset(); - return true; + else + { + //return empty array + while (!RespWriteUtils.TryWriteArrayLength(0, ref dcurr, dend)) + SendAndReset(); + return true; } // _ = ProcessOutputWithHeader(_output); diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 018503d28ef..ee29b318f2d 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -20,7 +20,6 @@ public class StreamObject : IDisposable StreamID lastId; long totalEntriesAdded; SingleWriterMultiReaderLock _lock; - private bool _disposed; /// /// Constructor @@ -30,7 +29,7 @@ public class StreamObject : IDisposable public StreamObject(string logDir, long pageSize, long memorySize, int safeTailRefreshFreqMs) { device = logDir == null ? new NullDevice() : Devices.CreateLogDevice("streamLogs/" + logDir + "/streamLog", preallocateFile: false); - log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize, SafeTailRefreshFrequencyMs = safeTailRefreshFreqMs}); + log = new TsavoriteLog(new TsavoriteLogSettings { LogDevice = device, PageSize = pageSize, MemorySize = memorySize, SafeTailRefreshFrequencyMs = safeTailRefreshFreqMs }); index = new BTree(device.SectorSize); totalEntriesAdded = 0; lastId = default; @@ -64,7 +63,7 @@ public void IncrementID(ref StreamID incrementedID) incrementedID.setMS(newMs); incrementedID.setSeq(newSeq); - + } /// @@ -104,9 +103,9 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) // parse user-defined ID // can be of following formats: - // 1. ts (seq = 0) - // 2. ts-* (auto-generate seq number) - // 3. ts-seq + // 1. ts (seq = 0) + // 2. ts-* (auto-generate seq number) + // 3. ts-seq // last character is a * if (*(idSlice.ptr + idSlice.length - 1) == '*') @@ -124,7 +123,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) { return false; } - + // check if timestamp is greater than last added entry's decoded ts if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { @@ -358,13 +357,13 @@ public bool ParseStreamIDFromString(string idString, out StreamID id) } if (!idString.Contains('-')) { - + if (!ulong.TryParse(idString, out ulong ms)) { return false; } id.setMS(ms); - id.setSeq(0); + id.setSeq(0); return true; } return ParseCompleteStreamIDFromString(idString, out id); diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 217b6b650e3..8655cb50bff 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -17,7 +17,6 @@ static unsafe void Main(string[] args) var tree = new BTree((uint)BTreeNode.PAGE_SIZE); ulong N = 400000; bool verbose = true; - bool sanity = false; if (args.Length > 0) { for (int i = 0; i < args.Length; i++) @@ -26,10 +25,6 @@ static unsafe void Main(string[] args) { verbose = true; } - else if (args[i] == "-s") - { - sanity = true; - } else if (args[i] == "-N") { N = ulong.Parse(args[i + 1]); @@ -148,7 +143,7 @@ static unsafe void Main(string[] args) // do a range query to check again tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); - Debug.Assert(tombstones.Count == 4); + Debug.Assert(tombstones.Count == 4); Console.WriteLine("Delete check passed "); // print all times collected in a csv format From 9d9c995f9a19039186fd4291a7abd3775a03efb3 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Mon, 26 May 2025 18:38:17 -0400 Subject: [PATCH 30/56] applied changes to propagate modified RESP3 output --- libs/server/Resp/RespServerSession.cs | 4 +- libs/server/Resp/StreamCommands.cs | 20 ++++----- libs/server/Stream/Stream.cs | 60 +++++++++++++++++---------- libs/server/Stream/StreamManager.cs | 14 ++++--- test/Garnet.test/RespStreamTests.cs | 7 ---- 5 files changed, 58 insertions(+), 47 deletions(-) diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index ad71693f3c1..15367a9ce39 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -902,10 +902,10 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st RespCommand.SDIFF => SetDiff(ref storageApi), RespCommand.SDIFFSTORE => SetDiffStore(ref storageApi), // Stream Commands - RespCommand.XADD => StreamAdd(), + RespCommand.XADD => StreamAdd(respProtocolVersion), RespCommand.XLEN => StreamLength(), RespCommand.XDEL => StreamDelete(), - RespCommand.XRANGE => StreamRange(), + RespCommand.XRANGE => StreamRange(respProtocolVersion), _ => ProcessOtherCommands(cmd, ref storageApi) }; return success; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 7ae68bb4473..a5f5b7b2c66 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -13,7 +13,7 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase /// STREAMADD /// /// - private unsafe bool StreamAdd() + private unsafe bool StreamAdd(byte respProtocolVersion) { if (parseState.Count < 4) { @@ -38,15 +38,16 @@ private unsafe bool StreamAdd() if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { - cachedStream.AddEntry(vPtr, vsize, idGiven, numPairs, ref _output); + cachedStream.AddEntry(vPtr, vsize, idGiven, numPairs, ref _output, respProtocolVersion); } else { - streamManager.StreamAdd(key, idGiven, vPtr, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream); + streamManager.StreamAdd(key, idGiven, vPtr, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream, respProtocolVersion); // since we added to a new stream that was not in the cache, try adding it to the cache sessionStreamCache.TryAddStreamToCache(lastStreamKey, lastStream); } - _ = ProcessOutputWithHeader(_output); + // _ = ProcessOutputWithHeader(_output); + ProcessOutput(_output); return true; } @@ -84,7 +85,7 @@ private bool StreamLength() /// STREAMRANGE /// /// - public unsafe bool StreamRange() + public unsafe bool StreamRange(byte respProtocolVersion) { // command is of format: XRANGE key start end [COUNT count] // we expect at least 3 arguments @@ -118,16 +119,17 @@ public unsafe bool StreamRange() // check if the stream exists in cache if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { - cachedStream.ReadRange(startId, endId, count, ref _output); + cachedStream.ReadRange(startId, endId, count, ref _output, respProtocolVersion); success = true; } else { - success = streamManager.StreamRange(key, startId, endId, count, ref _output); + success = streamManager.StreamRange(key, startId, endId, count, ref _output, respProtocolVersion); } if (success) { - _ = ProcessOutputWithHeader(_output); + // _ = ProcessOutputWithHeader(_output); + ProcessOutput(_output); } else { @@ -137,8 +139,6 @@ public unsafe bool StreamRange() return true; } - // _ = ProcessOutputWithHeader(_output); - return true; } diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index ee29b318f2d..317d6ca80d8 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -216,7 +216,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) /// /// byte array of the entry to store in the stream /// True if entry is added successfully - public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output) + public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output, byte respProtocolVersion) { byte* ptr = output.SpanByte.ToPointer(); var curr = ptr; @@ -225,15 +225,19 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int bool isMemory = false; byte* tmpPtr = null; StreamID id = default; + using var writer = new RespMemoryWriter(respProtocolVersion, ref output); // take a lock to ensure thread safety _lock.WriteLock(); + try { bool canParseID = parseIDString(idSlice, ref id); if (!canParseID) { - while (!RespWriteUtils.TryWriteError("ERR Syntax", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteError("ERR Syntax", ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteError("ERR Syntax"); + _lock.WriteUnlock(); return; } @@ -242,8 +246,10 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); if (!enqueueInLog) { - while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteError("ERR StreamAdd failed"); + _lock.WriteUnlock(); return; } @@ -253,8 +259,10 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int // bool added = true; if (!added) { - while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteError("ERR StreamAdd failed"); + _lock.WriteUnlock(); return; } // copy encoded ms and seq @@ -264,8 +272,9 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int totalEntriesAdded++; // write back the decoded ID of the entry added string idString = $"{id.getMS()}-{id.getSeq()}"; - while (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteSimpleString(idString); } } finally @@ -376,8 +385,9 @@ public bool ParseStreamIDFromString(string idString, out StreamID id) /// end of range /// threshold to scanning /// - public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndMemory output) + public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndMemory output, byte respProtocolVersion) { + using var writer = new RespMemoryWriter(respProtocolVersion, ref output); _lock.ReadLock(); try { @@ -431,8 +441,9 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM { // write length of how many entries we will print out - while (!RespWriteUtils.TryWriteArrayLength(count, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteArrayLength(count, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteArrayLength(count); byte* e; while (iter.GetNext(out var entry, out _, out long currentAddress, out long nextAddress)) @@ -468,16 +479,20 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM Span value = entryBytes.Slice(20); // we can already write back the ID that we read - while (!RespWriteUtils.TryWriteArrayLength(2, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - if (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) - { - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - } + // while (!RespWriteUtils.TryWriteArrayLength(2, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteArrayLength(2); + + // if (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) + // { + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // } + writer.WriteSimpleString(idString); // print array length for the number of key-value pairs in the entry - while (!RespWriteUtils.TryWriteArrayLength(numPairs, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteArrayLength(numPairs, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteArrayLength(numPairs); // write key-value pairs fixed (byte* p = value) @@ -493,8 +508,9 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM return; } var o = new Span(tmpPtr, tmpSize).ToArray(); - while (!RespWriteUtils.TryWriteBulkString(o, ref curr, end)) - ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // while (!RespWriteUtils.TryWriteBulkString(o, ref curr, end)) + // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + writer.WriteBulkString(o); read += (int)(e - orig); } } diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 7d8574ff7a7..49a2d34d6c4 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -36,7 +36,8 @@ public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) /// /// key of last stream accessed (for cache) /// reference to last stream accessed (for cache) - public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream) + /// RESP protocol version + public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream, byte respProtocolVersion) { // copy key store this key in the dictionary byte[] key = new byte[keySlice.Length]; @@ -52,7 +53,7 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, i foundStream = streams.TryGetValue(key, out stream); if (foundStream) { - stream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + stream.AddEntry(value, valueLength, idSlice, numPairs, ref output, respProtocolVersion); // update last accessed stream key lastStream = stream; streamKey = key; @@ -76,14 +77,14 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, i { // stream was not found with this key so create a new one StreamObject newStream = new StreamObject(null, defPageSize, defMemorySize, safeTailRefreshFreqMs); - newStream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + newStream.AddEntry(value, valueLength, idSlice, numPairs, ref output, respProtocolVersion); streams.TryAdd(key, newStream); streamKey = key; lastStream = newStream; } else { - stream.AddEntry(value, valueLength, idSlice, numPairs, ref output); + stream.AddEntry(value, valueLength, idSlice, numPairs, ref output, respProtocolVersion); lastStream = stream; streamKey = key; } @@ -127,7 +128,8 @@ public unsafe ulong StreamLength(ArgSlice keySlice) /// end of range /// threshold to limit scanning /// - public unsafe bool StreamRange(ArgSlice keySlice, string start, string end, int count, ref SpanByteAndMemory output) + /// RESP protocol version + public unsafe bool StreamRange(ArgSlice keySlice, string start, string end, int count, ref SpanByteAndMemory output, byte respProtocolVersion) { var key = keySlice.ToArray(); if (streams != null && streams.Count > 0) @@ -135,7 +137,7 @@ public unsafe bool StreamRange(ArgSlice keySlice, string start, string end, int bool foundStream = streams.TryGetValue(key, out StreamObject stream); if (foundStream) { - stream.ReadRange(start, end, count, ref output); + stream.ReadRange(start, end, count, ref output, respProtocolVersion); return true; } } diff --git a/test/Garnet.test/RespStreamTests.cs b/test/Garnet.test/RespStreamTests.cs index 78c636e7a9e..b2e6fea22eb 100644 --- a/test/Garnet.test/RespStreamTests.cs +++ b/test/Garnet.test/RespStreamTests.cs @@ -3,17 +3,10 @@ using System; using System.Collections.Generic; -using System.Globalization; using System.Linq; -using System.Text; -using System.Threading.Tasks; -using Embedded.server; -using Garnet.common; -using Garnet.server; using NUnit.Framework; using NUnit.Framework.Legacy; using StackExchange.Redis; -using Tsavorite.core; namespace Garnet.test { From 925dbc76d5c2a2b928b9766b35c68b4527437555 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Mon, 26 May 2025 21:15:49 -0400 Subject: [PATCH 31/56] added streams as an optional --- libs/host/Configuration/Options.cs | 5 +++++ libs/host/defaults.conf | 5 ++++- libs/server/Resp/RespServerSession.cs | 7 +++++-- libs/server/Servers/GarnetServerOptions.cs | 3 +++ libs/server/StoreWrapper.cs | 6 ++++-- 5 files changed, 21 insertions(+), 5 deletions(-) diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs index 2353c44f118..ba758729029 100644 --- a/libs/host/Configuration/Options.cs +++ b/libs/host/Configuration/Options.cs @@ -615,6 +615,10 @@ public IEnumerable LuaAllowedFunctions [Option("max-databases", Required = false, HelpText = "Max number of logical databases allowed in a single Garnet server instance")] public int MaxDatabases { get; set; } + [OptionValidation] + [Option("streams", Required = false, HelpText = "Enable streams on server.")] + public bool? EnableStreams { get; set; } + /// /// This property contains all arguments that were not parsed by the command line argument parser /// @@ -883,6 +887,7 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) UnixSocketPath = UnixSocketPath, UnixSocketPermission = unixSocketPermissions, MaxDatabases = MaxDatabases, + EnableStreams = EnableStreams.GetValueOrDefault(), }; } diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf index cf93dfe8021..9ac4d45c1ad 100644 --- a/libs/host/defaults.conf +++ b/libs/host/defaults.conf @@ -405,5 +405,8 @@ "UnixSocketPermission": 0, /* Max number of logical databases allowed in a single Garnet server instance */ - "MaxDatabases": 16 + "MaxDatabases": 16, + + /* Enable use of streams inside Garnet */ + "EnableStreams": false, } \ No newline at end of file diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 15367a9ce39..95954789331 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -274,8 +274,11 @@ public RespServerSession( } // grab stream manager from storeWrapper - this.streamManager = storeWrapper.streamManager; - sessionStreamCache = new SessionStreamCache(); + if (storeWrapper.serverOptions.EnableStreams) + { + this.streamManager = storeWrapper.streamManager; + sessionStreamCache = new SessionStreamCache(); + } } /// diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 581e8900ed4..a3c02389f6d 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -514,6 +514,9 @@ public string GetObjectStoreCheckpointDirectory(int dbId) => public string GetAppendOnlyFileDirectory(int dbId) => Path.Combine(AppendOnlyFileBaseDirectory, GetAppendOnlyFileDirectoryName(dbId)); + // Enable STREAMS on server + public bool EnableStreams = false; + /// /// Page size for BTree index for STREAM /// diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 105958e1cbe..57ee74ced50 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -260,9 +260,11 @@ public StoreWrapper(StoreWrapper storeWrapper, bool recordToAof) : this(storeWra clusterFactory: null, loggerFactory: storeWrapper.loggerFactory) { - // initialize stream manager - this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0); + if (serverOptions.EnableStreams) + { + this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0); + } } /// From ff74ba086c2bed404c8ba9b76c74ed83bdd77a7a Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Tue, 10 Jun 2025 13:08:42 -0400 Subject: [PATCH 32/56] formatting fixes --- libs/server/BTreeIndex/BTreeInsert.cs | 38 ++++++++++++------------ libs/server/BTreeIndex/BTreeInternals.cs | 2 +- libs/server/BTreeIndex/BTreeLookup.cs | 2 +- libs/server/Resp/Parser/RespCommand.cs | 4 +-- test/Garnet.test/BTreeTests.cs | 2 +- 5 files changed, 24 insertions(+), 24 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 239fb5f00ee..9e090d4c811 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -25,7 +25,7 @@ public bool Insert(byte* key, Value value) return InsertToLeafNode(ref leaf, ref rootToTailLeaf, key, value, true); } - public bool Insert(byte* key,ReadOnlySpan keySpan, Value value) + public bool Insert(byte* key, ReadOnlySpan keySpan, Value value) { BTreeNode* leaf = null; stats.totalFastInserts++; @@ -38,7 +38,7 @@ public bool Insert(byte* key,ReadOnlySpan keySpan, Value value) public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, byte* key, Value value, bool appendToLeaf = false) { int index; - if(appendToLeaf) + if (appendToLeaf) { // if leaf has space if (leaf->info->count < BTreeNode.LEAF_CAPACITY) @@ -53,7 +53,7 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse index = leaf->info->count; return SplitLeafNode(ref leaf, ref nodesTraversed, key, value, index); } - + // find the index where the key should be inserted index = leaf->LowerBound(key); if (index < leaf->info->count && BTreeNode.Compare(key, leaf->GetKey(index)) == 0) @@ -69,7 +69,7 @@ public bool InsertToLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraverse var sourceSpan = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, (leaf->info->count - index) * BTreeNode.KEY_SIZE); var destinationSpan = new Span(leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index) * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); - + leaf->SetKey(index, key); leaf->SetValue(index, value); leaf->info->count++; @@ -84,7 +84,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, var memoryBlock = (IntPtr*)NativeMemory.AlignedAlloc((nuint)BTreeNode.PAGE_SIZE, (nuint)BTreeNode.PAGE_SIZE); stats.numAllocates++; BTreeNode* newLeaf = BTreeNode.Create(BTreeNodeType.Leaf, memoryBlock); - + leaf->info->count = SPLIT_LEAF_POSITION; newLeaf->info->previous = leaf; newLeaf->info->next = leaf->info->next; @@ -108,7 +108,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, { // new key goes to the new leaf var newIndex = index - leaf->info->count; - + // move the keys from old node to the new node using ReadOnlySpan var sourceSpan = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, newIndex * BTreeNode.KEY_SIZE); var destinationSpan = new Span(newLeaf->keys, newIndex * BTreeNode.KEY_SIZE); @@ -125,7 +125,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, var newLeafValuesSpan = new Span(newLeaf->data.values, newIndex * sizeof(Value)); existingLeafValuesSpan.CopyTo(newLeafValuesSpan); newLeaf->SetValue(newIndex, value); - + var existingLeafValuesSpan2 = new ReadOnlySpan(leaf->data.values + index, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); var newLeafValuesSpan2 = new Span(newLeaf->data.values + newIndex + 1, (BTreeNode.LEAF_CAPACITY - index) * sizeof(Value)); existingLeafValuesSpan2.CopyTo(newLeafValuesSpan2); @@ -136,7 +136,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, var existingLeafKeysSpan = new ReadOnlySpan(leaf->keys + (leaf->info->count - 1) * BTreeNode.KEY_SIZE, newLeaf->info->count * BTreeNode.KEY_SIZE); var newLeafKeysSpan = new Span(newLeaf->keys, newLeaf->info->count * BTreeNode.KEY_SIZE); existingLeafKeysSpan.CopyTo(newLeafKeysSpan); - + var existingLeafKeysSpan2 = new ReadOnlySpan(leaf->keys + index * BTreeNode.KEY_SIZE, (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); var newLeafKeysSpan2 = new Span(leaf->keys + ((index + 1) * BTreeNode.KEY_SIZE), (leaf->info->count - index - 1) * BTreeNode.KEY_SIZE); existingLeafKeysSpan2.CopyTo(newLeafKeysSpan2); @@ -145,14 +145,14 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, var existingLeafValuesSpan = new ReadOnlySpan(leaf->data.values + leaf->info->count - 1, newLeaf->info->count * sizeof(Value)); var newLeafValuesSpan = new Span(newLeaf->data.values, newLeaf->info->count * sizeof(Value)); existingLeafValuesSpan.CopyTo(newLeafValuesSpan); - + var existingLeafValuesSpan2 = new ReadOnlySpan(leaf->data.values + index, (leaf->info->count - index - 1) * sizeof(Value)); var newLeafValuesSpan2 = new Span(leaf->data.values + index + 1, (leaf->info->count - index - 1) * sizeof(Value)); existingLeafValuesSpan2.CopyTo(newLeafValuesSpan2); leaf->SetValue(index, value); leaf->info->validCount++; } - + uint validCount = 0; // the leaf that is split will also be the tail node; so update the tail pointer if (leaf == tail) @@ -168,7 +168,7 @@ public bool SplitLeafNode(ref BTreeNode* leaf, ref BTreeNode*[] nodesTraversed, // update the parent node with the new key PushUpKeyInInternalNode(ref nodesTraversed, newLeaf->GetKey(0), ref newLeaf, SPLIT_INTERNAL_POSITION, validCount); return true; - } + } public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, ref BTreeNode* child, int splitPos, uint newValidCount) { @@ -211,7 +211,7 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r var sourceSpan = new ReadOnlySpan(node->keys + index * BTreeNode.KEY_SIZE, (node->info->count - index) * BTreeNode.KEY_SIZE); var destinationSpan = new Span(node->keys + ((index + 1) * BTreeNode.KEY_SIZE), (node->info->count - index) * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); - + // move all children starting from index+1 to the right using a for loop for (var j = node->info->count; j > index; j--) { @@ -260,16 +260,16 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + (nodeToSplit->info->count + 1) * BTreeNode.KEY_SIZE, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); var destinationSpan = new Span(newNode->keys, (index - nodeToSplit->info->count - 1) * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); - + var existingNodeKeysSpan = new ReadOnlySpan(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); var newNodeKeysSpan = new Span(newNode->keys + (index - nodeToSplit->info->count) * BTreeNode.KEY_SIZE, (BTreeNode.INTERNAL_CAPACITY - index) * BTreeNode.KEY_SIZE); existingNodeKeysSpan.CopyTo(newNodeKeysSpan); newNode->SetKey(index - nodeToSplit->info->count - 1, key); - + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + 1 + nodeToSplit->info->count, (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); var newNodeChildrenSpan = new Span(newNode->data.children, (index - nodeToSplit->info->count) * sizeof(BTreeNode*)); existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); - + var existingNodeChildrenSpan2 = new ReadOnlySpan(nodeToSplit->data.children + 1 + index, newNode->info->count * sizeof(BTreeNode*)); var newNodeChildrenSpan2 = new Span(newNode->data.children + 1 + index - nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); existingNodeChildrenSpan2.CopyTo(newNodeChildrenSpan2); @@ -281,7 +281,7 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); var destinationSpan = new Span(newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); - + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + 1 + nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); var newNodeChildrenSpan = new Span(newNode->data.children + 1, newNode->info->count * sizeof(BTreeNode*)); existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); @@ -293,16 +293,16 @@ public void InsertToInternalNodeWithinCapacity(ref BTreeNode* node, byte* key, r var sourceSpan = new ReadOnlySpan(nodeToSplit->keys + nodeToSplit->info->count * BTreeNode.KEY_SIZE, newNode->info->count * BTreeNode.KEY_SIZE); var destinationSpan = new Span(newNode->keys, newNode->info->count * BTreeNode.KEY_SIZE); sourceSpan.CopyTo(destinationSpan); - + var existingNodeKeysSpan = new ReadOnlySpan(nodeToSplit->keys + index * BTreeNode.KEY_SIZE, (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); var newNodeKeysSpan = new Span(nodeToSplit->keys + ((index + 1) * BTreeNode.KEY_SIZE), (nodeToSplit->info->count - index) * BTreeNode.KEY_SIZE); existingNodeKeysSpan.CopyTo(newNodeKeysSpan); nodeToSplit->SetKey(index, key); - + var existingNodeChildrenSpan = new ReadOnlySpan(nodeToSplit->data.children + nodeToSplit->info->count, newNode->info->count * sizeof(BTreeNode*)); var newNodeChildrenSpan = new Span(newNode->data.children, newNode->info->count * sizeof(BTreeNode*)); existingNodeChildrenSpan.CopyTo(newNodeChildrenSpan); - + var existingNodeChildrenSpan2 = new ReadOnlySpan(nodeToSplit->data.children + index + 1, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); var newNodeChildrenSpan2 = new Span(nodeToSplit->data.children + index + 2, (nodeToSplit->info->count - index + 1) * sizeof(BTreeNode*)); existingNodeChildrenSpan2.CopyTo(newNodeChildrenSpan2); diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index d7b243beb3e..4b79af5a114 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -142,7 +142,7 @@ public unsafe struct BTreeNode return node; } - + public byte* GetKey(int index) { byte* keyAddress = keys + (index * KEY_SIZE); diff --git a/libs/server/BTreeIndex/BTreeLookup.cs b/libs/server/BTreeIndex/BTreeLookup.cs index e5837a64327..bf73a5bbbae 100644 --- a/libs/server/BTreeIndex/BTreeLookup.cs +++ b/libs/server/BTreeIndex/BTreeLookup.cs @@ -142,7 +142,7 @@ public int Get(byte* start, byte* end, out Value startVal, out Value endVal, out i++; } } - + if (leaf == endLeaf || scanComplete) { break; diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 89c821e14ab..410fa731fbc 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -976,7 +976,7 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan return RespCommand.XDEL; } break; - + case 'Z': if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nZADD\r\n"u8)) { @@ -1330,7 +1330,7 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan return RespCommand.UNLINK; } break; - + case 'X': if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("XRANGE\r\n"u8)) { diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index c41cd5d9754..ce8d9843e52 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -46,7 +46,7 @@ public void InitBTreeLeafNode() // free the leaf BTree.FreeNode(ref leaf); - + leaf = null; } From 04acc5dbf07339a1178690531708abd80190e092 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Tue, 10 Jun 2025 13:28:22 -0400 Subject: [PATCH 33/56] minor edit --- libs/server/Stream/Stream.cs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 317d6ca80d8..d747e19d67e 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -218,11 +218,11 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) /// True if entry is added successfully public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output, byte respProtocolVersion) { - byte* ptr = output.SpanByte.ToPointer(); - var curr = ptr; - var end = curr + output.Length; - MemoryHandle ptrHandle = default; - bool isMemory = false; + // byte* ptr = output.SpanByte.ToPointer(); + // var curr = ptr; + // var end = curr + output.Length; + // MemoryHandle ptrHandle = default; + // bool isMemory = false; byte* tmpPtr = null; StreamID id = default; using var writer = new RespMemoryWriter(respProtocolVersion, ref output); @@ -281,8 +281,8 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int { // log.Commit(); - if (isMemory) ptrHandle.Dispose(); - output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); + // if (isMemory) ptrHandle.Dispose(); + // output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); _lock.WriteUnlock(); } From 859f07b684d91f3fea9e616b79b0eef951a2a962 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 15 Jun 2025 19:44:37 -0400 Subject: [PATCH 34/56] fixed minor bug in keeping streams optional --- libs/server/StoreWrapper.cs | 4 ++++ libs/server/Stream/Stream.cs | 7 ++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 57ee74ced50..7745ad9ce1c 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -242,6 +242,10 @@ public StoreWrapper( clusterProvider = clusterFactory.CreateClusterProvider(this); ctsCommit = new(); runId = Generator.CreateHexId(); + if (serverOptions.EnableStreams) + { + this.streamManager = new StreamManager(serverOptions.StreamPageSizeBytes(), serverOptions.StreamMemorySizeBytes(), 0); + } } /// diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index d747e19d67e..39f623d23bf 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -236,8 +236,9 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int { // while (!RespWriteUtils.TryWriteError("ERR Syntax", ref curr, end)) // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); + // _lock.WriteUnlock(); writer.WriteError("ERR Syntax"); - _lock.WriteUnlock(); + return; } @@ -249,7 +250,7 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteError("ERR StreamAdd failed"); - _lock.WriteUnlock(); + // _lock.WriteUnlock(); return; } @@ -262,7 +263,7 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteError("ERR StreamAdd failed"); - _lock.WriteUnlock(); + // _lock.WriteUnlock(); return; } // copy encoded ms and seq From 9415e906cd5866bc0e6722d6da4bcb1054f6a91e Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Sun, 15 Jun 2025 19:52:47 -0400 Subject: [PATCH 35/56] cleanup --- libs/server/Stream/Stream.cs | 55 +++--------------------------------- 1 file changed, 4 insertions(+), 51 deletions(-) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 39f623d23bf..f435f13b993 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -6,7 +6,6 @@ using Garnet.server.BTreeIndex; using Garnet.common; using System.Diagnostics; -using System.Buffers; using System.Runtime.CompilerServices; using System.Buffers.Binary; @@ -218,11 +217,6 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) /// True if entry is added successfully public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output, byte respProtocolVersion) { - // byte* ptr = output.SpanByte.ToPointer(); - // var curr = ptr; - // var end = curr + output.Length; - // MemoryHandle ptrHandle = default; - // bool isMemory = false; byte* tmpPtr = null; StreamID id = default; using var writer = new RespMemoryWriter(respProtocolVersion, ref output); @@ -234,11 +228,7 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int bool canParseID = parseIDString(idSlice, ref id); if (!canParseID) { - // while (!RespWriteUtils.TryWriteError("ERR Syntax", ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // _lock.WriteUnlock(); writer.WriteError("ERR Syntax"); - return; } @@ -247,23 +237,17 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); if (!enqueueInLog) { - // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteError("ERR StreamAdd failed"); - // _lock.WriteUnlock(); return; } var streamValue = new Value((ulong)retAddress); bool added = index.Insert((byte*)Unsafe.AsPointer(ref id.idBytes[0]), streamValue); - // bool added = true; + if (!added) { - // while (!RespWriteUtils.TryWriteError("ERR StreamAdd failed", ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteError("ERR StreamAdd failed"); - // _lock.WriteUnlock(); return; } // copy encoded ms and seq @@ -273,19 +257,13 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int totalEntriesAdded++; // write back the decoded ID of the entry added string idString = $"{id.getMS()}-{id.getSeq()}"; - // while (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteSimpleString(idString); } } finally { // log.Commit(); - - // if (isMemory) ptrHandle.Dispose(); - // output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); _lock.WriteUnlock(); - } } @@ -419,7 +397,6 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM { return; } - //endID.seq = long.MaxValue; endID.setSeq(long.MaxValue); } @@ -427,11 +404,6 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM startAddr = (long)startVal.address; endAddr = (long)endVal.address + 1; - byte* ptr = output.SpanByte.ToPointer(); - var curr = ptr; - var end = curr + output.Length; - MemoryHandle ptrHandle = default; - bool isMemory = false; byte* tmpPtr = null; int tmpSize = 0; long readCount = 0; @@ -441,9 +413,6 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM using (var iter = log.Scan(startAddr, endAddr, scanUncommitted: true)) { - // write length of how many entries we will print out - // while (!RespWriteUtils.TryWriteArrayLength(count, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteArrayLength(count); byte* e; @@ -472,27 +441,18 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM var idBytes = entryBytes.Slice(0, 16); var ts = BinaryPrimitives.ReadUInt64BigEndian(idBytes.Slice(0, 8)); var seq = BinaryPrimitives.ReadUInt64BigEndian(idBytes.Slice(8, 8)); - // var ts = BitConverter.ToUInt64(idBytes.Slice(0, 8)); - // var seq = BitConverter.ToUInt64(idBytes.Slice(8, 8)); + string idString = $"{ts}-{seq}"; Span numPairsBytes = entryBytes.Slice(16, 4); int numPairs = BitConverter.ToInt32(numPairsBytes); Span value = entryBytes.Slice(20); // we can already write back the ID that we read - // while (!RespWriteUtils.TryWriteArrayLength(2, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteArrayLength(2); - // if (!RespWriteUtils.TryWriteSimpleString(idString, ref curr, end)) - // { - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); - // } writer.WriteSimpleString(idString); // print array length for the number of key-value pairs in the entry - // while (!RespWriteUtils.TryWriteArrayLength(numPairs, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteArrayLength(numPairs); // write key-value pairs @@ -509,8 +469,6 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM return; } var o = new Span(tmpPtr, tmpSize).ToArray(); - // while (!RespWriteUtils.TryWriteBulkString(o, ref curr, end)) - // ObjectUtils.ReallocateOutput(ref output, ref isMemory, ref ptr, ref ptrHandle, ref curr, ref end); writer.WriteBulkString(o); read += (int)(e - orig); } @@ -524,10 +482,7 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM } } finally - { - if (isMemory) ptrHandle.Dispose(); - output.Length = (int)(curr - ptr) + sizeof(ObjectOutputHeader); - } + { } } finally { @@ -577,9 +532,7 @@ public void Dispose() device.Dispose(); } finally - { - - } + { } } } } \ No newline at end of file From 6b36d88d184e314080bc0ae672eaf4825c7cd65d Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Mon, 23 Jun 2025 13:56:20 -0400 Subject: [PATCH 36/56] updated XADD to return bulk string --- libs/server/Stream/Stream.cs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index f435f13b993..9773488b550 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -257,7 +257,10 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int totalEntriesAdded++; // write back the decoded ID of the entry added string idString = $"{id.getMS()}-{id.getSeq()}"; - writer.WriteSimpleString(idString); + // convert idString to ReadOnlySpan + var idSpan = System.Text.Encoding.UTF8.GetBytes(idString); + // write id as bulk string + writer.WriteBulkString(idSpan); } } finally From 3a89868046963be8f4e24f9eebd2ad087bee14cd Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 10:40:00 -0400 Subject: [PATCH 37/56] minor edits to error messages; used WriteAsciiBulkString for stream id output after XADD --- libs/server/Stream/Stream.cs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 9773488b550..777c148ee35 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -237,7 +237,7 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int bool enqueueInLog = log.TryEnqueueStreamEntry(id.idBytes, sizeof(StreamID), numPairs, value, valueLength, out long retAddress); if (!enqueueInLog) { - writer.WriteError("ERR StreamAdd failed"); + writer.WriteNull(); return; } @@ -247,7 +247,7 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int if (!added) { - writer.WriteError("ERR StreamAdd failed"); + writer.WriteNull(); return; } // copy encoded ms and seq @@ -257,10 +257,8 @@ public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int totalEntriesAdded++; // write back the decoded ID of the entry added string idString = $"{id.getMS()}-{id.getSeq()}"; - // convert idString to ReadOnlySpan - var idSpan = System.Text.Encoding.UTF8.GetBytes(idString); // write id as bulk string - writer.WriteBulkString(idSpan); + writer.WriteAsciiBulkString(idString); } } finally From f027ed6c2b75c1a6347a2333279cd60544bd48bc Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 12:50:16 -0400 Subject: [PATCH 38/56] added back code for BTreeTrim (work in progress) --- libs/server/BTreeIndex/BTreeTrim.cs | 305 ++++++++++++++++++++++++++++ playground/BTree/Program.cs | 10 +- 2 files changed, 314 insertions(+), 1 deletion(-) create mode 100644 libs/server/BTreeIndex/BTreeTrim.cs diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs new file mode 100644 index 00000000000..ee7d9522b9f --- /dev/null +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -0,0 +1,305 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; +using System.Runtime.CompilerServices; + +namespace Garnet.server.BTreeIndex +{ + public unsafe partial class BTree + { + public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + underflowingNodes = 0; + validKeysRemoved = 0; + numLeavesDeleted = 0; + + var nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; + BTreeNode* leaf = null; + TraverseToLeaf(ref leaf, ref nodesTraversed, key, out int[] internalSlots); + + // find index for key in leaf node - this returns the index of first key >= given key + var index = leaf->LowerBound(key); + // headValidKey = new ReadOnlySpan(leaf->GetKey(index), BTreeNode.KEY_SIZE); + // headValidValue = leaf->GetValue(index); + headValidKey = new byte[BTreeNode.KEY_SIZE]; + var headValidKeyPtr = leaf->GetKey(index); + Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + headValidValue = leaf->GetValue(index); + + // insert tombstones until index to mark as deleted + for (var i = 0; i < index; i++) + { + leaf->SetValueValid(i, false); + leaf->info->validCount--; + } + + if (leaf == head) + { + numLeavesDeleted = 0; + return; + } + + // traverse the leaf level to delete preceding leaf nodes + var node = leaf->info->previous; + var nodesToTraverseInSubtree = internalSlots[1] - 1; + uint deletedValidCount = 0; + while (node != null) + { + var validCount = node->info->validCount; + var count = node->info->count; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += validCount; + nodesToTraverseInSubtree--; + } + + var prev = node->info->previous; + if (prev == null) + { + Debug.Assert(node == head, "Head node should not have a previous node"); + } + + stats.numLeafNodes--; + stats.numKeys -= count; + stats.numValidKeys -= validCount; + validKeysRemoved += validCount; + + // deallocate the node + FreeNode(ref node); + numLeavesDeleted++; + + // continue iteration + node = prev; + } + + leaf->info->previous = null; + head = leaf; + + // traverse internal nodes except root and delete preceding internal nodes + for (int i = 1; i < stats.depth - 1; i++) + { + node = nodesTraversed[i]; + var slotOfKey = internalSlots[i]; + + if (slotOfKey > 0) + { + // shift children leftwards until slotOfKey (inclusive) using ReadOnlySpan + var sourceSpan = new ReadOnlySpan(node->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(node->keys, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + var sourceChildrenSpan = new ReadOnlySpan(node->data.children + (slotOfKey - 1) + 1, ((slotOfKey - 1)) * sizeof(BTreeNode*)); + var destinationChildrenSpan = new Span(node->data.children, ((slotOfKey - 1)) * sizeof(BTreeNode*)); + sourceChildrenSpan.CopyTo(destinationChildrenSpan); + } + var prevCount = node->info->count; + node->info->count -= slotOfKey; + nodesTraversed[i]->info->validCount -= deletedValidCount; + + if (prevCount > BTreeNode.INTERNAL_CAPACITY / 2 && node->info->count < BTreeNode.INTERNAL_CAPACITY / 2) + { + underflowingNodes++; + } + + deletedValidCount = 0; + node = nodesTraversed[i]->info->previous; + while (node != null) + { + var temp = node->info->previous; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += node->info->validCount; + nodesToTraverseInSubtree--; + } + FreeNode(ref node); + stats.numInternalNodes--; + node = temp; + } + nodesTraversed[i]->info->previous = null; + // corner case: slotOfKey points to last child => after deletion only one child remains + // delete all partent levels and re-assign root + if (i + 1 < stats.depth) + { + var nextSlot = internalSlots[i + 1]; + if (nextSlot == nodesTraversed[i]->info->count) + { + var newRoot = nodesTraversed[i]; + var originalDepth = stats.depth; + for (int j = i + 1; j < originalDepth; j++) + { + var curr = nodesTraversed[j]; + while (curr != null) + { + var pre = curr->info->previous; + FreeNode(ref curr); + stats.numInternalNodes--; + curr = pre; + } + stats.depth--; + } + root = newRoot; + break; + } + } + } + } + + public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nodesTraversed, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + var depth = stats.depth - 1; + ulong currentValidCount = 0; + var current = node; + int[] internalSlots = new int[MAX_TREE_DEPTH]; + int underflowingNodes = 0; + validKeysRemoved = 0; + numLeavesDeleted = 0; + headValidKey = new byte[BTreeNode.KEY_SIZE]; + + if (length >= stats.numValidKeys) + { + headValidValue = current->GetValue(0); + Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + return; + } + + while (depth > 0) + { + if (current->info->type == BTreeNodeType.Internal) + { + for (var i = current->info->count; i >= 0; i--) + { + var child = current->GetChild(i); + if (currentValidCount + child->info->validCount >= length) + { + nodesTraversed[depth] = child; + internalSlots[depth] = i; + current = child; + break; + } + else + { + currentValidCount += child->info->validCount; + } + } + } + depth--; + } + + headValidValue = current->GetValue(0); + Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + + var leaf = current->info->previous; + uint deletedValidCount = 0; + var nodesToTraverseInSubtree = internalSlots[depth + 1] - 1; + while (leaf != null) + { + var count = leaf->info->count; + var validCount = leaf->info->validCount; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += validCount; + nodesToTraverseInSubtree--; + } + var prev = leaf->info->previous; + if (prev == null) + { + Debug.Assert(leaf == head, "Head node should not have a previous node"); + } + stats.numLeafNodes--; + stats.numKeys -= count; + stats.numValidKeys -= validCount; + validKeysRemoved += validCount; + + // deallocate the node + FreeNode(ref leaf); + numLeavesDeleted++; + leaf = prev; + } + current->info->previous = null; + head = current; + // traverse the internal nodes except root and delete preceding internal nodes + for (int i = 1; i < stats.depth - 1; i++) + { + var slotOfKey = internalSlots[i]; + var inner = nodesTraversed[i]; + if (inner == null) + { + break; + } + if (slotOfKey > 0) + { + // shift keys and children from slotOfKey to beginning + var sourceSpan = new ReadOnlySpan(inner->keys + (slotOfKey - 1) * BTreeNode.KEY_SIZE, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); + var destinationSpan = new Span(inner->keys, ((slotOfKey - 1)) * BTreeNode.KEY_SIZE); + sourceSpan.CopyTo(destinationSpan); + + var sourceChildrenSpan = new ReadOnlySpan(inner->data.children + (slotOfKey - 1) + 1, ((slotOfKey - 1)) * sizeof(BTreeNode*)); + var destinationChildrenSpan = new Span(inner->data.children, ((slotOfKey - 1)) * sizeof(BTreeNode*)); + sourceChildrenSpan.CopyTo(destinationChildrenSpan); + } + var prevCount = inner->info->count; + inner->info->count -= slotOfKey; + nodesTraversed[i]->info->validCount -= deletedValidCount; + + if (prevCount > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info->count < BTreeNode.INTERNAL_CAPACITY / 2) + { + underflowingNodes++; + } + deletedValidCount = 0; + nodesToTraverseInSubtree = slotOfKey - 1; + inner = inner->info->previous; + while (inner != null && inner != root) + { + var temp = inner->info->previous; + if (nodesToTraverseInSubtree >= 0) + { + deletedValidCount += inner->info->validCount; + nodesToTraverseInSubtree--; + } + FreeNode(ref inner); + stats.numInternalNodes--; + inner = temp; + } + nodesTraversed[i]->info->previous = null; + // corner case: slotOfKey points to last child => after deletion only one child remains + // delete all parent levels and re-assign root + if (i + 1 < stats.depth) + { + var nextSlot = internalSlots[i + 1]; + if (nextSlot == nodesTraversed[i + 1]->info->count) + { + var newRoot = nodesTraversed[i]; + var originalDepth = stats.depth; + for (int j = i + 1; j < originalDepth; j++) + { + var curr = nodesTraversed[j]; + while (curr != null) + { + var pre = curr->info->previous; + FreeNode(ref curr); + stats.numInternalNodes--; + curr = pre; + } + stats.depth--; + } + root = newRoot; + break; + } + } + } + } + public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + int underflowingNodes; + TrimByID(key, out underflowingNodes, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + } + + public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + { + var nodesTraversed = new BTreeNode*[BTree.MAX_TREE_DEPTH]; + TrimByLength(ref root, length, ref nodesTraversed, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + } + } +} \ No newline at end of file diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 8655cb50bff..4d1e1826811 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -15,7 +15,7 @@ class Program static unsafe void Main(string[] args) { var tree = new BTree((uint)BTreeNode.PAGE_SIZE); - ulong N = 400000; + ulong N = 2000; bool verbose = true; if (args.Length > 0) { @@ -112,6 +112,14 @@ static unsafe void Main(string[] args) if (verbose) Console.WriteLine("Range query check passed "); + // tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + // Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); + // Console.WriteLine("num leaves deleted = " + numLeavesDeleted); + + // tree.TrimByLength(200, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); + // Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); + // Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); + // now let's delete some keys sw.Reset(); int num_deletes = 100; From c52bc73c20f036612a7b9eb4c109e88b9d2bbca3 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 14:43:46 -0400 Subject: [PATCH 39/56] fixed validCount bug in TrimByLength --- libs/server/BTreeIndex/BTree.cs | 14 +++++ libs/server/BTreeIndex/BTreeTrim.cs | 9 ++-- playground/BTree/Program.cs | 80 ++++++++++++++--------------- test/Garnet.test/BTreeTests.cs | 17 ++++++ 4 files changed, 77 insertions(+), 43 deletions(-) diff --git a/libs/server/BTreeIndex/BTree.cs b/libs/server/BTreeIndex/BTree.cs index 8ee6ca78f3c..2f60e392e60 100644 --- a/libs/server/BTreeIndex/BTree.cs +++ b/libs/server/BTreeIndex/BTree.cs @@ -96,6 +96,20 @@ public static void FreeNode(ref BTreeNode* node) } } + public static void Deallocate(ref BTreeNode* node) + { + // Free the memory handle + if (node->memoryHandle != null) + { + NativeMemory.Free(node->memoryHandle); + node->info = null; + node->keys = null; + node->data.values = null; + node->data.children = null; + node->memoryHandle = null; + } + } + /// /// Deallocates the memory allocated for the B+Tree /// diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index ee7d9522b9f..0593230fe8c 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -164,6 +164,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod return; } + nodesTraversed[depth] = current; while (depth > 0) { if (current->info->type == BTreeNodeType.Internal) @@ -173,7 +174,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod var child = current->GetChild(i); if (currentValidCount + child->info->validCount >= length) { - nodesTraversed[depth] = child; + nodesTraversed[depth - 1] = child; internalSlots[depth] = i; current = child; break; @@ -197,6 +198,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod { var count = leaf->info->count; var validCount = leaf->info->validCount; + if (nodesToTraverseInSubtree >= 0) { deletedValidCount += validCount; @@ -241,6 +243,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod } var prevCount = inner->info->count; inner->info->count -= slotOfKey; + // inner->info->validCount -= deletedValidCount; nodesTraversed[i]->info->validCount -= deletedValidCount; if (prevCount > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info->count < BTreeNode.INTERNAL_CAPACITY / 2) @@ -258,7 +261,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod deletedValidCount += inner->info->validCount; nodesToTraverseInSubtree--; } - FreeNode(ref inner); + Deallocate(ref inner); stats.numInternalNodes--; inner = temp; } @@ -278,7 +281,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod while (curr != null) { var pre = curr->info->previous; - FreeNode(ref curr); + Deallocate(ref curr); stats.numInternalNodes--; curr = pre; } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 4d1e1826811..a68e7ce0edb 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -15,7 +15,7 @@ class Program static unsafe void Main(string[] args) { var tree = new BTree((uint)BTreeNode.PAGE_SIZE); - ulong N = 2000; + ulong N = 50000; bool verbose = true; if (args.Length > 0) { @@ -116,49 +116,49 @@ static unsafe void Main(string[] args) // Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); // Console.WriteLine("num leaves deleted = " + numLeavesDeleted); - // tree.TrimByLength(200, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); - // Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); - // Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); + tree.TrimByLength(2000, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); + Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); + Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); // now let's delete some keys - sw.Reset(); - int num_deletes = 100; - int num_successfully_deleted = 0; - for (int i = 0; i < num_deletes; i++) - { - // generate a random index to delete - int idx = new Random().Next(0, (int)N); - sw.Start(); - bool val = false; - // bool val = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); - sw.Stop(); - if (val) - { - num_successfully_deleted++; - } - } - long deleteTime = (long)(sw.ElapsedTicks * nanosecondsPerTick); - if (verbose) - { - Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); - Console.WriteLine("Time for deletion = " + deleteTime + " ns"); - } + // sw.Reset(); + // int num_deletes = 100; + // int num_successfully_deleted = 0; + // for (int i = 0; i < num_deletes; i++) + // { + // // generate a random index to delete + // int idx = new Random().Next(0, (int)N); + // sw.Start(); + // bool val = false; + // // bool val = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); + // sw.Stop(); + // if (val) + // { + // num_successfully_deleted++; + // } + // } + // long deleteTime = (long)(sw.ElapsedTicks * nanosecondsPerTick); + // if (verbose) + // { + // Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); + // Console.WriteLine("Time for deletion = " + deleteTime + " ns"); + // } - tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); - tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); - tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); - tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); + // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); - // do a range query to check again - tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); - Debug.Assert(tombstones.Count == 4); - Console.WriteLine("Delete check passed "); + // // do a range query to check again + // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); + // Debug.Assert(tombstones.Count == 4); + // Console.WriteLine("Delete check passed "); - // print all times collected in a csv format - Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); - tree.Deallocate(); - Console.WriteLine("Num allocates = " + tree.stats.numAllocates); - Console.WriteLine("Num deallocates = " + tree.stats.numDeallocates); - Console.WriteLine("All checks passed"); + // // print all times collected in a csv format + // Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); + // tree.Deallocate(); + // Console.WriteLine("Num allocates = " + tree.stats.numAllocates); + // Console.WriteLine("Num deallocates = " + tree.stats.numDeallocates); + // Console.WriteLine("All checks passed"); } } \ No newline at end of file diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index ce8d9843e52..736f8489bc0 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -131,5 +131,22 @@ public void Delete() ClassicAssert.AreEqual(tree.ValidCount, N - delCount); tree.Deallocate(); } + + [Test] + [Category("Trim")] + public void TrimByLength() + { + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); + } + + ulong trimLength = 5000; // trim the tree to half its size + tree.TrimByLength(trimLength, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + ClassicAssert.GreaterOrEqual(N - trimLength, validKeysRemoved); + + tree.Deallocate(); + } } } From 2b34ff6eebbbc8e480e7b0e03c246da36369f746 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 20:10:33 -0400 Subject: [PATCH 40/56] fuxed bug on validCount propagation + trimByID --- libs/server/BTreeIndex/BTreeInsert.cs | 1 + libs/server/BTreeIndex/BTreeTrim.cs | 23 +++++++++++++++-------- playground/BTree/Program.cs | 12 ++++++------ 3 files changed, 22 insertions(+), 14 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeInsert.cs b/libs/server/BTreeIndex/BTreeInsert.cs index 9e090d4c811..d9073dbd930 100644 --- a/libs/server/BTreeIndex/BTreeInsert.cs +++ b/libs/server/BTreeIndex/BTreeInsert.cs @@ -194,6 +194,7 @@ public void PushUpKeyInInternalNode(ref BTreeNode*[] nodesTraversed, byte* key, } // split internal node + node->info->validCount += newValidCount; var newNode = SplitInternalNode(ref node, ref nodesTraversed, ref key, ref child, splitPos, index, i); if (rootToTailLeaf[i] == node && tail != head && BTreeNode.Compare(key, tailMinKey) <= 0) { diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 0593230fe8c..9579a358432 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -33,6 +33,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe { leaf->SetValueValid(i, false); leaf->info->validCount--; + validKeysRemoved++; } if (leaf == head) @@ -44,7 +45,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe // traverse the leaf level to delete preceding leaf nodes var node = leaf->info->previous; var nodesToTraverseInSubtree = internalSlots[1] - 1; - uint deletedValidCount = 0; + uint deletedValidCount = (uint)(leaf->info->count - leaf->info->validCount); while (node != null) { var validCount = node->info->validCount; @@ -67,7 +68,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe validKeysRemoved += validCount; // deallocate the node - FreeNode(ref node); + Deallocate(ref node); numLeavesDeleted++; // continue iteration @@ -77,6 +78,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe leaf->info->previous = null; head = leaf; + bool rootReassigned = false; // traverse internal nodes except root and delete preceding internal nodes for (int i = 1; i < stats.depth - 1; i++) { @@ -96,24 +98,24 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } var prevCount = node->info->count; node->info->count -= slotOfKey; - nodesTraversed[i]->info->validCount -= deletedValidCount; + node->info->validCount -= deletedValidCount; if (prevCount > BTreeNode.INTERNAL_CAPACITY / 2 && node->info->count < BTreeNode.INTERNAL_CAPACITY / 2) { underflowingNodes++; } - deletedValidCount = 0; node = nodesTraversed[i]->info->previous; + // deletedValidCount = 0; while (node != null) { var temp = node->info->previous; if (nodesToTraverseInSubtree >= 0) { - deletedValidCount += node->info->validCount; + // deletedValidCount += node->info->validCount; nodesToTraverseInSubtree--; } - FreeNode(ref node); + Deallocate(ref node); stats.numInternalNodes--; node = temp; } @@ -133,17 +135,22 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe while (curr != null) { var pre = curr->info->previous; - FreeNode(ref curr); + Deallocate(ref curr); stats.numInternalNodes--; curr = pre; } stats.depth--; } root = newRoot; + rootReassigned = true; break; } } } + if (!rootReassigned && stats.depth > 1 && nodesTraversed[stats.depth - 1] != null) + { + nodesTraversed[stats.depth - 1]->info->validCount -= deletedValidCount; + } } public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nodesTraversed, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) @@ -215,7 +222,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod validKeysRemoved += validCount; // deallocate the node - FreeNode(ref leaf); + Deallocate(ref leaf); numLeavesDeleted++; leaf = prev; } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index a68e7ce0edb..fe55f33acf0 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -112,13 +112,13 @@ static unsafe void Main(string[] args) if (verbose) Console.WriteLine("Range query check passed "); - // tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); - // Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); - // Console.WriteLine("num leaves deleted = " + numLeavesDeleted); + tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); + Console.WriteLine("num leaves deleted = " + numLeavesDeleted); - tree.TrimByLength(2000, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); - Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); - Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); + // tree.TrimByLength(2000, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); + // Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); + // Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); // now let's delete some keys // sw.Reset(); From 91f93c2bf024c2436a47048b130e7df54602417b Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 21:46:04 -0400 Subject: [PATCH 41/56] minor bug fix; added BTreeTrimTests --- libs/server/BTreeIndex/BTreeTrim.cs | 10 ++-- playground/BTree/Program.cs | 78 ++++++++++++++--------------- test/Garnet.test/BTreeTests.cs | 25 +++++++-- 3 files changed, 67 insertions(+), 46 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 9579a358432..7c9ead74c35 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -46,6 +46,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe var node = leaf->info->previous; var nodesToTraverseInSubtree = internalSlots[1] - 1; uint deletedValidCount = (uint)(leaf->info->count - leaf->info->validCount); + var totalDeletedValidCount = deletedValidCount; while (node != null) { var validCount = node->info->validCount; @@ -55,6 +56,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe deletedValidCount += validCount; nodesToTraverseInSubtree--; } + totalDeletedValidCount += validCount; var prev = node->info->previous; if (prev == null) @@ -106,13 +108,13 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } node = nodesTraversed[i]->info->previous; - // deletedValidCount = 0; + deletedValidCount = 0; while (node != null) { var temp = node->info->previous; if (nodesToTraverseInSubtree >= 0) { - // deletedValidCount += node->info->validCount; + deletedValidCount += node->info->validCount; nodesToTraverseInSubtree--; } Deallocate(ref node); @@ -125,7 +127,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe if (i + 1 < stats.depth) { var nextSlot = internalSlots[i + 1]; - if (nextSlot == nodesTraversed[i]->info->count) + if (nextSlot == nodesTraversed[i + 1]->info->count) { var newRoot = nodesTraversed[i]; var originalDepth = stats.depth; @@ -149,7 +151,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } if (!rootReassigned && stats.depth > 1 && nodesTraversed[stats.depth - 1] != null) { - nodesTraversed[stats.depth - 1]->info->validCount -= deletedValidCount; + nodesTraversed[stats.depth - 1]->info->validCount -= totalDeletedValidCount; } } diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index fe55f33acf0..3d09968249f 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -112,53 +112,53 @@ static unsafe void Main(string[] args) if (verbose) Console.WriteLine("Range query check passed "); - tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); - Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); - Console.WriteLine("num leaves deleted = " + numLeavesDeleted); + // tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + // Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); + // Console.WriteLine("num leaves deleted = " + numLeavesDeleted); // tree.TrimByLength(2000, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); // Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); // Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); // now let's delete some keys - // sw.Reset(); - // int num_deletes = 100; - // int num_successfully_deleted = 0; - // for (int i = 0; i < num_deletes; i++) - // { - // // generate a random index to delete - // int idx = new Random().Next(0, (int)N); - // sw.Start(); - // bool val = false; - // // bool val = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); - // sw.Stop(); - // if (val) - // { - // num_successfully_deleted++; - // } - // } - // long deleteTime = (long)(sw.ElapsedTicks * nanosecondsPerTick); - // if (verbose) - // { - // Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); - // Console.WriteLine("Time for deletion = " + deleteTime + " ns"); - // } + sw.Reset(); + int num_deletes = 100; + int num_successfully_deleted = 0; + for (int i = 0; i < num_deletes; i++) + { + // generate a random index to delete + int idx = new Random().Next(0, (int)N); + sw.Start(); + bool val = false; + // bool val = tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[idx].idBytes[0])); + sw.Stop(); + if (val) + { + num_successfully_deleted++; + } + } + long deleteTime = (long)(sw.ElapsedTicks * nanosecondsPerTick); + if (verbose) + { + Console.WriteLine("Number of keys deleted = " + num_successfully_deleted); + Console.WriteLine("Time for deletion = " + deleteTime + " ns"); + } - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); - // tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 400].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 300].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 200].idBytes[0])); + tree.Delete((byte*)Unsafe.AsPointer(ref streamIDs[N - 100].idBytes[0])); - // // do a range query to check again - // tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); - // Debug.Assert(tombstones.Count == 4); - // Console.WriteLine("Delete check passed "); + // do a range query to check again + tree.Get((byte*)Unsafe.AsPointer(ref streamIDs[N - 500].idBytes[0]), (byte*)Unsafe.AsPointer(ref streamIDs[N - 1].idBytes[0]), out Value startVal1, out Value endVal1, out List tombstones); + Debug.Assert(tombstones.Count == 4); + Console.WriteLine("Delete check passed "); - // // print all times collected in a csv format - // Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); - // tree.Deallocate(); - // Console.WriteLine("Num allocates = " + tree.stats.numAllocates); - // Console.WriteLine("Num deallocates = " + tree.stats.numDeallocates); - // Console.WriteLine("All checks passed"); + // print all times collected in a csv format + Console.WriteLine(insertion_time + ", " + query_time + ", " + range_query_times[0] + ", " + range_query_times[1] + ", " + range_query_times[2] + ", " + deleteTime); + tree.Deallocate(); + Console.WriteLine("Num allocates = " + tree.stats.numAllocates); + Console.WriteLine("Num deallocates = " + tree.stats.numDeallocates); + Console.WriteLine("All checks passed"); } } \ No newline at end of file diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 736f8489bc0..8a80030c2ae 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -142,9 +142,28 @@ public void TrimByLength() tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); } - ulong trimLength = 5000; // trim the tree to half its size - tree.TrimByLength(trimLength, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); - ClassicAssert.GreaterOrEqual(N - trimLength, validKeysRemoved); + var trimLength = 5000; // trim the tree to half its size + tree.TrimByLength((ulong)trimLength, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + var validKeysRemaining = tree.RootValidCount + tree.TailValidCount; + ClassicAssert.GreaterOrEqual(validKeysRemaining, trimLength); + + tree.Deallocate(); + } + + [Test] + [Category("TrimByID")] + public void TrimByID() + { + var tree = new BTree((uint)BTreeNode.PAGE_SIZE); + for (ulong i = 0; i < N; i++) + { + tree.Insert((byte*)Unsafe.AsPointer(ref streamIDs[i].idBytes[0]), new Value(streamIDs[i].ms)); + } + + var streamIDToTrim = streamIDs[N - 1000]; + tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDToTrim.idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + var validKeysRemaining = tree.RootValidCount + tree.TailValidCount; + ClassicAssert.GreaterOrEqual((ulong)validKeysRemaining, N - validKeysRemoved); tree.Deallocate(); } From ca7d283f37cf49d77632ea2fe26def1a08cb0f59 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 21:55:57 -0400 Subject: [PATCH 42/56] return headValidKey as ReadOnlySpan --- libs/server/BTreeIndex/BTreeTrim.cs | 28 ++++++++++++++-------------- playground/BTree/Program.cs | 4 ++-- test/Garnet.test/BTreeTests.cs | 4 ++-- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 7c9ead74c35..662c6a7adc2 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -3,13 +3,12 @@ using System; using System.Diagnostics; -using System.Runtime.CompilerServices; namespace Garnet.server.BTreeIndex { public unsafe partial class BTree { - public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { underflowingNodes = 0; validKeysRemoved = 0; @@ -21,11 +20,11 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe // find index for key in leaf node - this returns the index of first key >= given key var index = leaf->LowerBound(key); - // headValidKey = new ReadOnlySpan(leaf->GetKey(index), BTreeNode.KEY_SIZE); + headValidKey = new ReadOnlySpan(leaf->GetKey(index), BTreeNode.KEY_SIZE); // headValidValue = leaf->GetValue(index); - headValidKey = new byte[BTreeNode.KEY_SIZE]; - var headValidKeyPtr = leaf->GetKey(index); - Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + // headValidKey = new byte[BTreeNode.KEY_SIZE]; + // var headValidKeyPtr = leaf->GetKey(index); + // Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); headValidValue = leaf->GetValue(index); // insert tombstones until index to mark as deleted @@ -155,7 +154,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } } - public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nodesTraversed, out ulong validKeysRemoved, out Value headValidValue, out byte[] headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ref BTreeNode* node, ulong length, out ulong validKeysRemoved, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { var depth = stats.depth - 1; ulong currentValidCount = 0; @@ -164,12 +163,13 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod int underflowingNodes = 0; validKeysRemoved = 0; numLeavesDeleted = 0; - headValidKey = new byte[BTreeNode.KEY_SIZE]; + headValidKey = default; + BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; if (length >= stats.numValidKeys) { headValidValue = current->GetValue(0); - Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + headValidKey = new ReadOnlySpan(current->GetKey(0), BTreeNode.KEY_SIZE); return; } @@ -198,7 +198,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod } headValidValue = current->GetValue(0); - Buffer.MemoryCopy(current->GetKey(0), Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); + headValidKey = new ReadOnlySpan(current->GetKey(0), BTreeNode.KEY_SIZE); var leaf = current->info->previous; uint deletedValidCount = 0; @@ -302,16 +302,16 @@ public void TrimByLength(ref BTreeNode* node, ulong length, ref BTreeNode*[] nod } } } - public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { int underflowingNodes; TrimByID(key, out underflowingNodes, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); } - public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { - var nodesTraversed = new BTreeNode*[BTree.MAX_TREE_DEPTH]; - TrimByLength(ref root, length, ref nodesTraversed, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + + TrimByLength(ref root, length, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); } } } \ No newline at end of file diff --git a/playground/BTree/Program.cs b/playground/BTree/Program.cs index 3d09968249f..6fbb87d6d24 100644 --- a/playground/BTree/Program.cs +++ b/playground/BTree/Program.cs @@ -112,11 +112,11 @@ static unsafe void Main(string[] args) if (verbose) Console.WriteLine("Range query check passed "); - // tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + // tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDs[500].idBytes[0]), out var validKeysRemoved, out var headValue, out var headValidKey, out var numLeavesDeleted); // Console.WriteLine("Trimmed by ID: validKeysRemoved = " + validKeysRemoved); // Console.WriteLine("num leaves deleted = " + numLeavesDeleted); - // tree.TrimByLength(2000, out ulong validKeysRemoved2, out Value headValue2, out byte[] headValidKey2, out uint numLeavesDeleted2); + // tree.TrimByLength(2000, out var validKeysRemoved2, out var headValue2, out var headValidKey2, out var numLeavesDeleted2); // Console.WriteLine("Trimmed by length: validKeysRemoved = " + validKeysRemoved2); // Console.WriteLine("num leaves deleted = " + numLeavesDeleted2); diff --git a/test/Garnet.test/BTreeTests.cs b/test/Garnet.test/BTreeTests.cs index 8a80030c2ae..8854d30fe15 100644 --- a/test/Garnet.test/BTreeTests.cs +++ b/test/Garnet.test/BTreeTests.cs @@ -143,7 +143,7 @@ public void TrimByLength() } var trimLength = 5000; // trim the tree to half its size - tree.TrimByLength((ulong)trimLength, out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + tree.TrimByLength((ulong)trimLength, out var validKeysRemoved, out var headValue, out var headValidKey, out var numLeavesDeleted); var validKeysRemaining = tree.RootValidCount + tree.TailValidCount; ClassicAssert.GreaterOrEqual(validKeysRemaining, trimLength); @@ -161,7 +161,7 @@ public void TrimByID() } var streamIDToTrim = streamIDs[N - 1000]; - tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDToTrim.idBytes[0]), out ulong validKeysRemoved, out Value headValue, out byte[] headValidKey, out uint numLeavesDeleted); + tree.TrimByID((byte*)Unsafe.AsPointer(ref streamIDToTrim.idBytes[0]), out var validKeysRemoved, out var headValue, out var headValidKey, out var numLeavesDeleted); var validKeysRemaining = tree.RootValidCount + tree.TailValidCount; ClassicAssert.GreaterOrEqual((ulong)validKeysRemaining, N - validKeysRemoved); From 4ff5394bb02fb530aa60aa5163869555f6507c90 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Wed, 25 Jun 2025 21:57:24 -0400 Subject: [PATCH 43/56] cleanup --- libs/server/BTreeIndex/BTreeTrim.cs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 662c6a7adc2..45115af1368 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -21,10 +21,6 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe // find index for key in leaf node - this returns the index of first key >= given key var index = leaf->LowerBound(key); headValidKey = new ReadOnlySpan(leaf->GetKey(index), BTreeNode.KEY_SIZE); - // headValidValue = leaf->GetValue(index); - // headValidKey = new byte[BTreeNode.KEY_SIZE]; - // var headValidKeyPtr = leaf->GetKey(index); - // Buffer.MemoryCopy(headValidKeyPtr, Unsafe.AsPointer(ref headValidKey[0]), BTreeNode.KEY_SIZE, BTreeNode.KEY_SIZE); headValidValue = leaf->GetValue(index); // insert tombstones until index to mark as deleted @@ -252,7 +248,6 @@ public void TrimByLength(ref BTreeNode* node, ulong length, out ulong validKeysR } var prevCount = inner->info->count; inner->info->count -= slotOfKey; - // inner->info->validCount -= deletedValidCount; nodesTraversed[i]->info->validCount -= deletedValidCount; if (prevCount > BTreeNode.INTERNAL_CAPACITY / 2 && inner->info->count < BTreeNode.INTERNAL_CAPACITY / 2) From 0961e1b456980accc0fdd72f3ea579b010f57993 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 09:42:42 -0400 Subject: [PATCH 44/56] connecting XTRIM to Streams API --- libs/resources/RespCommandsDocs.json | 367 ++++++++++++++++-- libs/resources/RespCommandsInfo.json | 130 ++++--- libs/server/Resp/CmdStrings.cs | 1 + libs/server/Resp/Parser/RespCommand.cs | 7 + libs/server/Resp/RespServerSession.cs | 1 + libs/server/Resp/StreamCommands.cs | 41 ++ libs/server/Stream/Stream.cs | 53 +++ libs/server/Stream/StreamManager.cs | 18 + .../CommandInfoUpdater/SupportedCommand.cs | 1 + 9 files changed, 519 insertions(+), 100 deletions(-) diff --git a/libs/resources/RespCommandsDocs.json b/libs/resources/RespCommandsDocs.json index fa14521386e..de902ff48b0 100644 --- a/libs/resources/RespCommandsDocs.json +++ b/libs/resources/RespCommandsDocs.json @@ -1917,7 +1917,6 @@ } ] }, - { "Command": "DISCARD", "Name": "DISCARD", @@ -7663,6 +7662,290 @@ } ] }, + { + "Command": "XADD", + "Name": "XADD", + "Summary": "Appends a new message to a stream. Creates the key if it doesn\u0027t exist.", + "Group": "Stream", + "Complexity": "O(1) when adding a new entry, O(N) when trimming where N being the number of entries evicted.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NOMKSTREAM", + "DisplayText": "nomkstream", + "Type": "PureToken", + "Token": "NOMKSTREAM", + "ArgumentFlags": "Optional" + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "TRIM", + "Type": "Block", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "STRATEGY", + "Type": "OneOf", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MAXLEN", + "DisplayText": "maxlen", + "Type": "PureToken", + "Token": "MAXLEN" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MINID", + "DisplayText": "minid", + "Type": "PureToken", + "Token": "MINID" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "OPERATOR", + "Type": "OneOf", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "EQUAL", + "DisplayText": "equal", + "Type": "PureToken", + "Token": "=" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "APPROXIMATELY", + "DisplayText": "approximately", + "Type": "PureToken", + "Token": "~" + } + ] + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "THRESHOLD", + "DisplayText": "threshold", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "COUNT", + "DisplayText": "count", + "Type": "Integer", + "Token": "LIMIT", + "ArgumentFlags": "Optional" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "ID-SELECTOR", + "Type": "OneOf", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "AUTO-ID", + "DisplayText": "auto-id", + "Type": "PureToken", + "Token": "*" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "ID", + "DisplayText": "id", + "Type": "String" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "DATA", + "Type": "Block", + "ArgumentFlags": "Multiple", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "FIELD", + "DisplayText": "field", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "VALUE", + "DisplayText": "value", + "Type": "String" + } + ] + } + ] + }, + { + "Command": "XDEL", + "Name": "XDEL", + "Summary": "Returns the number of messages after removing them from a stream.", + "Group": "Stream", + "Complexity": "O(1) for each single item to delete in the stream, regardless of the stream size.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "ID", + "DisplayText": "id", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] + }, + { + "Command": "XLEN", + "Name": "XLEN", + "Summary": "Return the number of messages in a stream.", + "Group": "Stream", + "Complexity": "O(1)", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + } + ] + }, + { + "Command": "XRANGE", + "Name": "XRANGE", + "Summary": "Returns the messages from a stream within a range of IDs.", + "Group": "Stream", + "Complexity": "O(N) with N being the number of elements being returned. If N is constant (e.g. always asking for the first 10 elements with COUNT), you can consider it O(1).", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "START", + "DisplayText": "start", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "END", + "DisplayText": "end", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "COUNT", + "DisplayText": "count", + "Type": "Integer", + "Token": "COUNT", + "ArgumentFlags": "Optional" + } + ] + }, + { + "Command": "XTRIM", + "Name": "XTRIM", + "Summary": "Deletes messages from the beginning of a stream.", + "Group": "Stream", + "Complexity": "O(N), with N being the number of evicted entries. Constant times are very small however, since entries are organized in macro nodes containing multiple entries that can be released with a single deallocation.", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandKeyArgument", + "Name": "KEY", + "DisplayText": "key", + "Type": "Key", + "KeySpecIndex": 0 + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "TRIM", + "Type": "Block", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "STRATEGY", + "Type": "OneOf", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MAXLEN", + "DisplayText": "maxlen", + "Type": "PureToken", + "Token": "MAXLEN" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MINID", + "DisplayText": "minid", + "Type": "PureToken", + "Token": "MINID" + } + ] + }, + { + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "OPERATOR", + "Type": "OneOf", + "ArgumentFlags": "Optional", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "EQUAL", + "DisplayText": "equal", + "Type": "PureToken", + "Token": "=" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "APPROXIMATELY", + "DisplayText": "approximately", + "Type": "PureToken", + "Token": "~" + } + ] + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "THRESHOLD", + "DisplayText": "threshold", + "Type": "String" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "COUNT", + "DisplayText": "count", + "Type": "Integer", + "Token": "LIMIT", + "ArgumentFlags": "Optional" + } + ] + } + ] + }, { "Command": "ZADD", "Name": "ZADD", @@ -7775,6 +8058,12 @@ } ] }, + { + "Command": "ZCOLLECT", + "Name": "ZCOLLECT", + "Summary": "Manually trigger deletion of expired members from memory for SortedSet", + "Group": "Hash" + }, { "Command": "ZCOUNT", "Name": "ZCOUNT", @@ -7803,12 +8092,6 @@ } ] }, - { - "Command": "ZCOLLECT", - "Name": "ZCOLLECT", - "Summary": "Manually trigger deletion of expired members from memory for SortedSet", - "Group": "Hash" - }, { "Command": "ZDIFF", "Name": "ZDIFF", @@ -8597,11 +8880,11 @@ ] }, { - "Command": "ZPTTL", - "Name": "ZPTTL", - "Summary": "Returns the TTL in milliseconds of a sorted set member.", + "Command": "ZPOPMAX", + "Name": "ZPOPMAX", + "Summary": "Returns the highest-scoring members from a sorted set after removing them. Deletes the sorted set if the last member was popped.", "Group": "SortedSet", - "Complexity": "O(N) where N is the number of specified members", + "Complexity": "O(log(N)*M) with N being the number of elements in the sorted set, and M being the number of elements popped.", "Arguments": [ { "TypeDiscriminator": "RespCommandKeyArgument", @@ -8611,32 +8894,18 @@ "KeySpecIndex": 0 }, { - "TypeDiscriminator": "RespCommandContainerArgument", - "Name": "MEMBERS", - "Type": "Block", - "Token": "MEMBERS", - "Arguments": [ - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "NUMMEMBERS", - "DisplayText": "nummembers", - "Type": "Integer" - }, - { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "MEMBER", - "DisplayText": "member", - "Type": "String", - "ArgumentFlags": "Multiple" - } - ] + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "COUNT", + "DisplayText": "count", + "Type": "Integer", + "ArgumentFlags": "Optional" } ] }, { - "Command": "ZPOPMAX", - "Name": "ZPOPMAX", - "Summary": "Returns the highest-scoring members from a sorted set after removing them. Deletes the sorted set if the last member was popped.", + "Command": "ZPOPMIN", + "Name": "ZPOPMIN", + "Summary": "Returns the lowest-scoring members from a sorted set after removing them. Deletes the sorted set if the last member was popped.", "Group": "SortedSet", "Complexity": "O(log(N)*M) with N being the number of elements in the sorted set, and M being the number of elements popped.", "Arguments": [ @@ -8657,11 +8926,11 @@ ] }, { - "Command": "ZPOPMIN", - "Name": "ZPOPMIN", - "Summary": "Returns the lowest-scoring members from a sorted set after removing them. Deletes the sorted set if the last member was popped.", + "Command": "ZPTTL", + "Name": "ZPTTL", + "Summary": "Returns the TTL in milliseconds of a sorted set member.", "Group": "SortedSet", - "Complexity": "O(log(N)*M) with N being the number of elements in the sorted set, and M being the number of elements popped.", + "Complexity": "O(N) where N is the number of specified members", "Arguments": [ { "TypeDiscriminator": "RespCommandKeyArgument", @@ -8671,11 +8940,25 @@ "KeySpecIndex": 0 }, { - "TypeDiscriminator": "RespCommandBasicArgument", - "Name": "COUNT", - "DisplayText": "count", - "Type": "Integer", - "ArgumentFlags": "Optional" + "TypeDiscriminator": "RespCommandContainerArgument", + "Name": "MEMBERS", + "Type": "Block", + "Token": "MEMBERS", + "Arguments": [ + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "NUMMEMBERS", + "DisplayText": "nummembers", + "Type": "Integer" + }, + { + "TypeDiscriminator": "RespCommandBasicArgument", + "Name": "MEMBER", + "DisplayText": "member", + "Type": "String", + "ArgumentFlags": "Multiple" + } + ] } ] }, diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index c8bfe53eabe..dc5731db737 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -5018,7 +5018,6 @@ { "Command": "XADD", "Name": "XADD", - "IsInternal": false, "Arity": -5, "Flags": "DenyOom, Fast, Write", "FirstKey": 1, @@ -5043,20 +5042,17 @@ "Notes": "UPDATE instead of INSERT because of the optional trimming feature", "Flags": "RW, Update" } - ], - "SubCommands": null + ] }, { "Command": "XDEL", "Name": "XDEL", - "IsInternal": false, "Arity": -3, "Flags": "Fast, Write", "FirstKey": 1, "LastKey": 1, "Step": 1, "AclCategories": "Fast, Stream, Write", - "Tips": null, "KeySpecifications": [ { "BeginSearch": { @@ -5069,23 +5065,19 @@ "KeyStep": 1, "Limit": 0 }, - "Notes": null, "Flags": "RW, Delete" } - ], - "SubCommands": null + ] }, { "Command": "XLEN", "Name": "XLEN", - "IsInternal": false, "Arity": 2, "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, "AclCategories": "Fast, Read, Stream", - "Tips": null, "KeySpecifications": [ { "BeginSearch": { @@ -5098,23 +5090,19 @@ "KeyStep": 1, "Limit": 0 }, - "Notes": null, "Flags": "RO" } - ], - "SubCommands": null + ] }, { "Command": "XRANGE", "Name": "XRANGE", - "IsInternal": false, "Arity": -4, "Flags": "ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, "AclCategories": "Read, Slow, Stream", - "Tips": null, "KeySpecifications": [ { "BeginSearch": { @@ -5127,11 +5115,37 @@ "KeyStep": 1, "Limit": 0 }, - "Notes": null, "Flags": "RO, Access" } + ] + }, + { + "Command": "XTRIM", + "Name": "XTRIM", + "Arity": -4, + "Flags": "Write", + "FirstKey": 1, + "LastKey": 1, + "Step": 1, + "AclCategories": "Slow, Stream, Write", + "Tips": [ + "nondeterministic_output" ], - "SubCommands": null + "KeySpecifications": [ + { + "BeginSearch": { + "TypeDiscriminator": "BeginSearchIndex", + "Index": 1 + }, + "FindKeys": { + "TypeDiscriminator": "FindKeysRange", + "LastKey": 0, + "KeyStep": 1, + "Limit": 0 + }, + "Flags": "RW, Delete" + } + ] }, { "Command": "ZADD", @@ -5184,14 +5198,14 @@ ] }, { - "Command": "ZCOUNT", - "Name": "ZCOUNT", - "Arity": 4, - "Flags": "Fast, ReadOnly", + "Command": "ZCOLLECT", + "Name": "ZCOLLECT", + "Arity": 2, + "Flags": "Admin, Write", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, Read, SortedSet", + "AclCategories": "Admin, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5204,19 +5218,19 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RO, Access" + "Flags": "RW, Access, Update" } ] }, { - "Command": "ZCOLLECT", - "Name": "ZCOLLECT", - "Arity": 2, - "Flags": "Admin, Write", + "Command": "ZCOUNT", + "Name": "ZCOUNT", + "Arity": 4, + "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Write, Admin, Garnet", + "AclCategories": "Fast, Read, SortedSet", "KeySpecifications": [ { "BeginSearch": { @@ -5229,7 +5243,7 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RW, Access, Update" + "Flags": "RO, Access" } ] }, @@ -5301,7 +5315,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5326,7 +5340,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5351,7 +5365,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5555,7 +5569,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5580,7 +5594,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5605,7 +5619,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Write, Garnet", + "AclCategories": "Fast, SortedSet, Write, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5630,7 +5644,7 @@ "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5648,14 +5662,14 @@ ] }, { - "Command": "ZPTTL", - "Name": "ZPTTL", - "Arity": -5, - "Flags": "Fast, ReadOnly", + "Command": "ZPOPMAX", + "Name": "ZPOPMAX", + "Arity": -2, + "Flags": "Fast, Write", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, SortedSet, Write", "KeySpecifications": [ { "BeginSearch": { @@ -5668,13 +5682,13 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RO, Access" + "Flags": "RW, Access, Delete" } ] }, { - "Command": "ZPOPMAX", - "Name": "ZPOPMAX", + "Command": "ZPOPMIN", + "Name": "ZPOPMIN", "Arity": -2, "Flags": "Fast, Write", "FirstKey": 1, @@ -5698,14 +5712,14 @@ ] }, { - "Command": "ZPOPMIN", - "Name": "ZPOPMIN", - "Arity": -2, - "Flags": "Fast, Write", + "Command": "ZPTTL", + "Name": "ZPTTL", + "Arity": -5, + "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, SortedSet, Write", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { @@ -5718,7 +5732,7 @@ "KeyStep": 1, "Limit": 0 }, - "Flags": "RW, Access, Delete" + "Flags": "RO, Access" } ] }, @@ -6117,14 +6131,14 @@ ] }, { - "Command": "ZTTL", - "Name": "ZTTL", - "Arity": -5, + "Command": "ZSCORE", + "Name": "ZSCORE", + "Arity": 3, "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "SortedSet, Fast, Read, Garnet", + "AclCategories": "Fast, Read, SortedSet", "KeySpecifications": [ { "BeginSearch": { @@ -6142,14 +6156,14 @@ ] }, { - "Command": "ZSCORE", - "Name": "ZSCORE", - "Arity": 3, + "Command": "ZTTL", + "Name": "ZTTL", + "Arity": -5, "Flags": "Fast, ReadOnly", "FirstKey": 1, "LastKey": 1, "Step": 1, - "AclCategories": "Fast, Read, SortedSet", + "AclCategories": "Fast, Read, SortedSet, Garnet", "KeySpecifications": [ { "BeginSearch": { diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index 40d73540f44..8ed73a4043f 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -292,6 +292,7 @@ static partial class CmdStrings public static ReadOnlySpan RESP_ERR_XLEN_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xlen' command"u8; public static ReadOnlySpan RESP_ERR_XRANGE_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xrange' command"u8; public static ReadOnlySpan RESP_ERR_XDEL_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xdel' command"u8; + public static ReadOnlySpan RESP_ERR_XTRIM_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xtrim' command"u8; /// /// Response string templates diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 410fa731fbc..705c89784dd 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -198,6 +198,7 @@ public enum RespCommand : ushort XLEN, XRANGE, XDEL, + XTRIM, ZADD, ZCOLLECT, ZDIFFSTORE, @@ -1143,6 +1144,12 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan return RespCommand.WATCH; } break; + case 'X': + if (*(ulong*)(ptr + 3) == MemoryMarshal.Read("\nXTRIM\r\n"u8)) + { + return RespCommand.XTRIM; + } + break; case 'Z': if (*(ulong*)(ptr + 3) == MemoryMarshal.Read("\nZCARD\r\n"u8)) diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 95954789331..2aa436135b7 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -909,6 +909,7 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st RespCommand.XLEN => StreamLength(), RespCommand.XDEL => StreamDelete(), RespCommand.XRANGE => StreamRange(respProtocolVersion), + RespCommand.XTRIM => StreamTrim(), _ => ProcessOtherCommands(cmd, ref storageApi) }; return success; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index a5f5b7b2c66..c79386ad610 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -187,5 +187,46 @@ public bool StreamDelete() return true; } + public bool StreamTrim() + { + if (parseState.Count < 3) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_XTRIM_WRONG_NUM_ARGS); + } + + var key = parseState.GetArgSliceByRef(0); + var trimType = parseState.GetArgSliceByRef(1).ToString().ToUpper(); + var trimArg = parseState.GetArgSliceByRef(2); + + ulong validKeysRemoved = 0; + StreamTrimOpts optType = StreamTrimOpts.NONE; + switch (trimType) + { + case "MAXLEN": + optType = StreamTrimOpts.MAXLEN; + break; + case "MINID": + optType = StreamTrimOpts.MINID; + break; + } + + bool result; + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) + { + result = cachedStream.Trim(trimArg, optType, out validKeysRemoved); + } + else + { + result = streamManager.StreamTrim(key, trimArg, optType, out validKeysRemoved); + } + if (!result) + { + return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); + } + while (!RespWriteUtils.TryWriteInt64((long)validKeysRemoved, ref dcurr, dend)) + SendAndReset(); + return true; + } + } } \ No newline at end of file diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 777c148ee35..a80458b1c29 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -11,6 +11,13 @@ namespace Garnet.server { + public enum StreamTrimOpts + { + MAXLEN, + MINID, + NONE + } + public class StreamObject : IDisposable { readonly IDevice device; @@ -491,6 +498,52 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM } } + public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong validKeysRemoved) + { + uint numLeavesDeleted = 0; + Value headValue = default; + _lock.WriteLock(); + try + { + switch (optType) + { + case StreamTrimOpts.MAXLEN: + if (!RespReadUtils.ReadUlong(out ulong maxLen, ref trimArg.ptr, trimArg.ptr + trimArg.length)) + { + validKeysRemoved = 0; + return false; + } + index.TrimByLength(maxLen, out validKeysRemoved, out headValue, out var headValidKey, out numLeavesDeleted); + break; + case StreamTrimOpts.MINID: + if (!parseCompleteID(trimArg, out StreamID minID)) + { + validKeysRemoved = 0; + return false; + } + index.TrimByID((byte*)Unsafe.AsPointer(ref minID.idBytes[0]), out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + break; + default: + validKeysRemoved = 0; + break; + } + + if (numLeavesDeleted == 0) + { + // didn't delete any leaf nodes so done here + return true; + } + // truncate log to new head + var newHeadAddress = (long)headValue.address; + log.TruncateUntil(newHeadAddress); + } + finally + { + _lock.WriteUnlock(); + } + return true; + } + unsafe bool parseCompleteID(ArgSlice idSlice, out StreamID streamID) { diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 49a2d34d6c4..34231600411 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -170,6 +170,24 @@ public bool StreamDelete(ArgSlice keySlice, ArgSlice idSlice, out StreamObject l return false; } + public bool StreamTrim(ArgSlice keySlice, ArgSlice trimArg, StreamTrimOpts optType, out ulong validKeysRemoved) + { + bool foundStream; + var key = keySlice.ToArray(); + StreamObject stream; + validKeysRemoved = 0; + if (streams != null) + { + foundStream = streams.TryGetValue(key, out stream); + + if (foundStream) + { + return stream.Trim(trimArg, optType, out validKeysRemoved); + } + } + return true; // no keys removed so return true + } + /// public void Dispose() { diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index aac094a182d..d90fb4ae193 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -313,6 +313,7 @@ public class SupportedCommand new("XDEL", RespCommand.XDEL), new("XLEN", RespCommand.XLEN), new("XRANGE", RespCommand.XRANGE), + new("XTRIM", RespCommand.XTRIM), new("ZADD", RespCommand.ZADD), new("ZCARD", RespCommand.ZCARD), new("ZCOUNT", RespCommand.ZCOUNT), From d6f4e69cefc00f7a023ccc9d1f674af99ec5d30e Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 10:06:56 -0400 Subject: [PATCH 45/56] added XTrimTests --- test/Garnet.test/Resp/ACL/RespCommandTests.cs | 19 +++++++++++++++ test/Garnet.test/RespStreamTests.cs | 23 ++++++++++++++++++- test/Garnet.test/TestUtils.cs | 4 +++- 3 files changed, 44 insertions(+), 2 deletions(-) diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index e79b89d3fb1..8574cd03443 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -6482,6 +6482,25 @@ async Task DoXDelAsync(GarnetClient client) } } + public async Task XTRIMACLsAsync() + { + await CheckCommandsAsync( + "XTRIM", + [DoXTrimMinIDAsync, DoXTrimMaxLenAsync] + ); + async Task DoXTrimMinIDAsync(GarnetClient client) + { + long val = await client.ExecuteForLongResultAsync("XTRIM", ["foo", "MINID", "0-0"]); + ClassicAssert.AreEqual(0, val); + } + + async Task DoXTrimMaxLenAsync(GarnetClient client) + { + long val = await client.ExecuteForLongResultAsync("XTRIM", ["foo", "MAXLEN", "0"]); + ClassicAssert.AreEqual(0, val); + } + } + [Test] public async Task ZAddACLsAsync() { diff --git a/test/Garnet.test/RespStreamTests.cs b/test/Garnet.test/RespStreamTests.cs index b2e6fea22eb..51e24663fd0 100644 --- a/test/Garnet.test/RespStreamTests.cs +++ b/test/Garnet.test/RespStreamTests.cs @@ -22,7 +22,7 @@ public class RespStreamTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, lowMemory: true, enableStreams: true); server.Start(); random = new Random(); @@ -197,6 +197,27 @@ public void StreamDeleteMultipleTest() ClassicAssert.AreEqual(delCount, indices.Count); } + [Test] + [Category("Trim")] + public void StreamTrimMaxLenTest() + { + using var redis = ConnectionMultiplexer.Connect(TestUtils.GetConfig()); + var db = redis.GetDatabase(0); + + var streamKey = "trimByMaxLen"; + long count = 500; + for (long i = 0; i < count; i++) + { + var entryKey = GenerateRandomString(4); // generate random ascii string of length 4 + var entryValue = GenerateRandomString(4); // generate random ascii string of length 4 + var retId = db.StreamAdd(streamKey, entryKey, entryValue, $"{i + 1}-0"); + } + var maxLen = 100; + var trimCount = db.StreamTrim(streamKey, maxLen); + ClassicAssert.GreaterOrEqual(trimCount, 1); + ClassicAssert.GreaterOrEqual(count - trimCount, maxLen); + } + #endregion } diff --git a/test/Garnet.test/TestUtils.cs b/test/Garnet.test/TestUtils.cs index 7cda43beebe..f22ed5bdad5 100644 --- a/test/Garnet.test/TestUtils.cs +++ b/test/Garnet.test/TestUtils.cs @@ -259,7 +259,8 @@ public static GarnetServer CreateGarnetServer( UnixFileMode unixSocketPermission = default, int slowLogThreshold = 0, TextWriter logTo = null, - bool enableCluster = false) + bool enableCluster = false, + bool enableStreams = false) { if (useAzureStorage) IgnoreIfNotRunningAzureTests(); @@ -344,6 +345,7 @@ public static GarnetServer CreateGarnetServer( UnixSocketPath = unixSocketPath, UnixSocketPermission = unixSocketPermission, SlowLogThreshold = slowLogThreshold, + EnableStreams = enableStreams, }; if (!string.IsNullOrEmpty(pubSubPageSize)) From dfc2963aa2054cd0a337bae3ace3fbaf40f6ac0e Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 10:19:42 -0400 Subject: [PATCH 46/56] stream commands error out in case streams is disabled --- libs/server/Resp/StreamCommands.cs | 40 ++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index c79386ad610..fd8e71bcbfa 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -35,6 +35,14 @@ private unsafe bool StreamAdd(byte respProtocolVersion) int vsize = (int)(recvBufferPtr + endReadHead - vPtr); SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var disabledStreams = streamManager == null; + if (disabledStreams) + { + while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + SendAndReset(); + return true; + } + if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { @@ -66,6 +74,14 @@ private bool StreamLength() ulong streamLength; + var disabledStreams = streamManager == null; + if (disabledStreams) + { + while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + SendAndReset(); + return true; + } + // check if the stream exists in cache if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { @@ -114,6 +130,14 @@ public unsafe bool StreamRange(byte respProtocolVersion) SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); + var disabledStreams = streamManager == null; + if (disabledStreams) + { + while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + SendAndReset(); + return true; + } + bool success = false; // check if the stream exists in cache @@ -155,6 +179,14 @@ public bool StreamDelete() var key = parseState.GetArgSliceByRef(0); int deletedCount = 0; + var disabledStreams = streamManager == null; + if (disabledStreams) + { + while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + SendAndReset(); + return true; + } + // for every id, parse and delete the stream entry for (int i = 1; i < parseState.Count; i++) { @@ -210,6 +242,14 @@ public bool StreamTrim() break; } + var disabledStreams = streamManager == null; + if (disabledStreams) + { + while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + SendAndReset(); + return true; + } + bool result; if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { From da0c52f852253c704d53ce10f63f058bd7ae631b Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 11:27:32 -0400 Subject: [PATCH 47/56] cleanup + switched to ReadOnlySpan instead of byte[] where applicable --- libs/server/Resp/StreamCommands.cs | 28 ++++++++++++------- libs/server/Stream/Stream.cs | 4 +-- libs/server/Stream/StreamManager.cs | 2 +- .../cs/src/core/TsavoriteLog/TsavoriteLog.cs | 13 ++++----- 4 files changed, 27 insertions(+), 20 deletions(-) diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index fd8e71bcbfa..21cd63db64b 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using Garnet.common; using Tsavorite.core; @@ -10,9 +11,9 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase { readonly StreamManager streamManager; /// - /// STREAMADD + /// Adds a new entry to the stream. /// - /// + /// true if stream was added successfully; error otherwise private unsafe bool StreamAdd(byte respProtocolVersion) { if (parseState.Count < 4) @@ -31,8 +32,8 @@ private unsafe bool StreamAdd(byte respProtocolVersion) // grab the rest of the input that will mainly be k-v pairs as entry to the stream. byte* vPtr = parseState.GetArgSliceByRef(2).ptr - sizeof(int); - //int vsize = (int)(recvBufferPtr + bytesRead - vPtr); int vsize = (int)(recvBufferPtr + endReadHead - vPtr); + var streamDataSpan = new ReadOnlySpan(vPtr, vsize); SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); var disabledStreams = streamManager == null; @@ -46,23 +47,22 @@ private unsafe bool StreamAdd(byte respProtocolVersion) if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { - cachedStream.AddEntry(vPtr, vsize, idGiven, numPairs, ref _output, respProtocolVersion); + cachedStream.AddEntry(streamDataSpan, vsize, idGiven, numPairs, ref _output, respProtocolVersion); } else { - streamManager.StreamAdd(key, idGiven, vPtr, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream, respProtocolVersion); + streamManager.StreamAdd(key, idGiven, streamDataSpan, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream, respProtocolVersion); // since we added to a new stream that was not in the cache, try adding it to the cache sessionStreamCache.TryAddStreamToCache(lastStreamKey, lastStream); } - // _ = ProcessOutputWithHeader(_output); ProcessOutput(_output); return true; } /// - /// STREAMLENGTH + /// Retrieves the length of the stream. /// - /// + /// true if stream length was retrieved successfully; error otherwise private bool StreamLength() { if (parseState.Count != 1) @@ -98,9 +98,9 @@ private bool StreamLength() } /// - /// STREAMRANGE + /// Retrieves a range of stream entries. /// - /// + /// true if range of stream entries were retrieved successfully; error otherwise public unsafe bool StreamRange(byte respProtocolVersion) { // command is of format: XRANGE key start end [COUNT count] @@ -166,6 +166,10 @@ public unsafe bool StreamRange(byte respProtocolVersion) return true; } + /// + /// Deletes stream entry(s). + /// + /// true if stream entry(s) was deleted successfully; error otherwise public bool StreamDelete() { // command is of format: XDEL key id [id ...] @@ -219,6 +223,10 @@ public bool StreamDelete() return true; } + /// + /// Trims the stream to the specified length or ID. + /// + /// returns true if stream was trimmed successfully; error otherwise public bool StreamTrim() { if (parseState.Count < 3) diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index a80458b1c29..ce0e21924cf 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -222,7 +222,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) /// /// byte array of the entry to store in the stream /// True if entry is added successfully - public unsafe void AddEntry(byte* value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output, byte respProtocolVersion) + public unsafe void AddEntry(ReadOnlySpan value, int valueLength, ArgSlice idSlice, int numPairs, ref SpanByteAndMemory output, byte respProtocolVersion) { byte* tmpPtr = null; StreamID id = default; @@ -458,7 +458,7 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM // we can already write back the ID that we read writer.WriteArrayLength(2); - writer.WriteSimpleString(idString); + writer.WriteAsciiBulkString(idString); // print array length for the number of key-value pairs in the entry writer.WriteArrayLength(numPairs); diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 34231600411..6430a6830dd 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -37,7 +37,7 @@ public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) /// key of last stream accessed (for cache) /// reference to last stream accessed (for cache) /// RESP protocol version - public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, byte* value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream, byte respProtocolVersion) + public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, ReadOnlySpan value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream, byte respProtocolVersion) { // copy key store this key in the dictionary byte[] key = new byte[keySlice.Length]; diff --git a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs index 70490cdf775..0f75d51e82a 100644 --- a/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs +++ b/libs/storage/Tsavorite/cs/src/core/TsavoriteLog/TsavoriteLog.cs @@ -844,7 +844,7 @@ public unsafe bool TryEnqueue(ReadOnlySpan entry, out long logicalAddress) return true; } - public unsafe bool TryEnqueueStreamEntry(byte* id, int idLength, int numPairs, byte* entry, int entryLength, out long logicalAddress) + public unsafe bool TryEnqueueStreamEntry(byte* id, int idLength, int numPairs, ReadOnlySpan entry, int entryLength, out long logicalAddress) { logicalAddress = 0; var length = idLength + sizeof(int) + entryLength; @@ -865,15 +865,14 @@ public unsafe bool TryEnqueueStreamEntry(byte* id, int idLength, int numPairs, b var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); // start writing - // first we copy the id + // copy the id *(long*)(headerSize + physicalAddress) = *(long*)id; *(long*)(headerSize + physicalAddress + 8) = *(long*)(id + sizeof(long)); - // Buffer.MemoryCopy(id, (void*)(headerSize + physicalAddress), idLength, idLength); - // then we copy the number of pairs - // Buffer.MemoryCopy(numPairsBytes, (void*)(headerSize + physicalAddress + idLength), numPairsBytesLength, numPairsBytesLength); + // copy the number of pairs *(int*)(headerSize + physicalAddress + idLength) = numPairs; - // then we copy the entry - Buffer.MemoryCopy(entry, (void*)(headerSize + physicalAddress + idLength + sizeof(int)), entryLength, entryLength); + // copy the entry + fixed (byte* bp = &entry.GetPinnableReference()) + Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress + idLength + sizeof(int)), entryLength, entryLength); SetHeader(length, (byte*)physicalAddress); safeTailRefreshEntryEnqueued?.Signal(); From 4d946f52a02c4dfd52fd34cdca9990ea69adbdd1 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 11:45:03 -0400 Subject: [PATCH 48/56] matched error messages to Redis documentation --- libs/server/Resp/CmdStrings.cs | 6 ++++- libs/server/Resp/StreamCommands.cs | 10 +++---- libs/server/Stream/Stream.cs | 43 +++++++++++++++++++----------- 3 files changed, 37 insertions(+), 22 deletions(-) diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index 8ed73a4043f..d2837cbccdc 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -288,7 +288,11 @@ static partial class CmdStrings public static ReadOnlySpan RESP_ERR_FLUSHALL_READONLY_REPLICA => "ERR You can't write against a read only replica."u8; public static ReadOnlySpan RESP_ERR_DEUBG_DISALLOWED => @"ERR DEBUG command not allowed. If the EnableDebugCommand option is set to ""local"", you can run it from a local connection, otherwise you need to set this option in the configuration file, and then restart the server."u8; - public static ReadOnlySpan RESP_ERR_ZSET_MEMBER => "ERR could not decode requested zset member"u8; public static ReadOnlySpan RESP_ERR_XADD_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xadd' command"u8; + public static ReadOnlySpan RESP_ERR_ZSET_MEMBER => "ERR could not decode requested zset member"u8; + public static ReadOnlySpan RESP_ERR_STREAMS_DISABLED => "ERR STREAMS is disabled, enable it with --streams option."u8; + public static ReadOnlySpan RESP_ERR_XADD_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xadd' command"u8; + public static ReadOnlySpan RESP_ERR_XADD_INVALID_STREAM_ID => "ERR Invalid stream ID specified as stream command argument"u8; + public static ReadOnlySpan RESP_ERR_XADD_ID_NOT_GREATER => "ERR The ID specified in XADD is equal or smaller than the target stream top item"u8; public static ReadOnlySpan RESP_ERR_XLEN_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xlen' command"u8; public static ReadOnlySpan RESP_ERR_XRANGE_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xrange' command"u8; public static ReadOnlySpan RESP_ERR_XDEL_WRONG_NUM_ARGS => "ERR wrong number of arguments for 'xdel' command"u8; diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 21cd63db64b..60655768acb 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -39,7 +39,7 @@ private unsafe bool StreamAdd(byte respProtocolVersion) var disabledStreams = streamManager == null; if (disabledStreams) { - while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_STREAMS_DISABLED, ref dcurr, dend)) SendAndReset(); return true; } @@ -77,7 +77,7 @@ private bool StreamLength() var disabledStreams = streamManager == null; if (disabledStreams) { - while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_STREAMS_DISABLED, ref dcurr, dend)) SendAndReset(); return true; } @@ -133,7 +133,7 @@ public unsafe bool StreamRange(byte respProtocolVersion) var disabledStreams = streamManager == null; if (disabledStreams) { - while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_STREAMS_DISABLED, ref dcurr, dend)) SendAndReset(); return true; } @@ -186,7 +186,7 @@ public bool StreamDelete() var disabledStreams = streamManager == null; if (disabledStreams) { - while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_STREAMS_DISABLED, ref dcurr, dend)) SendAndReset(); return true; } @@ -253,7 +253,7 @@ public bool StreamTrim() var disabledStreams = streamManager == null; if (disabledStreams) { - while (!RespWriteUtils.TryWriteError("ERR STREAMS is disabled, enable it with --streams option."u8, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_STREAMS_DISABLED, ref dcurr, dend)) SendAndReset(); return true; } diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index ce0e21924cf..a395d963153 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -18,6 +18,13 @@ public enum StreamTrimOpts NONE } + public enum ParsedStreamEntryID + { + VALID, + INVALID, + NOT_GREATER, + } + public class StreamObject : IDisposable { readonly IDevice device; @@ -95,14 +102,13 @@ public unsafe void GenerateNextID(ref StreamID id) IncrementID(ref id); } - // TODO: implement this using parseState functions without operating with RespReadUtils - unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) + unsafe ParsedStreamEntryID parseIDString(ArgSlice idSlice, ref StreamID id) { // if we have to auto-generate the whole ID if (*idSlice.ptr == '*' && idSlice.length == 1) { GenerateNextID(ref id); - return true; + return ParsedStreamEntryID.VALID; } var lastIdDecodedTs = lastId.getMS(); @@ -119,7 +125,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) // has to be of format ts-*, check if '-' is the preceding character if (*(idSlice.ptr + idSlice.length - 2) != '-') { - return false; + return ParsedStreamEntryID.INVALID; } // parse the timestamp // slice the id to remove the last two characters @@ -127,13 +133,13 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) var idEnd = idSlice.ptr + idSlice.length - 2; if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idEnd)) { - return false; + return ParsedStreamEntryID.INVALID; } // check if timestamp is greater than last added entry's decoded ts if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { - return false; + return ParsedStreamEntryID.NOT_GREATER; } else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { @@ -162,12 +168,12 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) { if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + idSlice.length)) { - return false; + return ParsedStreamEntryID.INVALID; } // check if timestamp is greater than last added entry if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { - return false; + return ParsedStreamEntryID.NOT_GREATER; } else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { @@ -187,24 +193,24 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) var slicedSeq = new ArgSlice(idSlice.ptr + index + 1, idSlice.length - index - 1); if (!RespReadUtils.ReadUlong(out ulong timestamp, ref idSlice.ptr, idSlice.ptr + index)) { - return false; + return ParsedStreamEntryID.INVALID; } var seqBegin = idSlice.ptr + index + 1; var seqEnd = idSlice.ptr + idSlice.length; if (!RespReadUtils.ReadUlong(out ulong seq, ref seqBegin, seqEnd)) { - return false; + return ParsedStreamEntryID.INVALID; } if (totalEntriesAdded != 0 && timestamp < lastIdDecodedTs) { - return false; + return ParsedStreamEntryID.NOT_GREATER; } else if (totalEntriesAdded != 0 && timestamp == lastIdDecodedTs) { if (seq <= lastId.seq) { - return false; + return ParsedStreamEntryID.INVALID; } } // use ID and seq given by user @@ -214,7 +220,7 @@ unsafe bool parseIDString(ArgSlice idSlice, ref StreamID id) } } - return true; + return ParsedStreamEntryID.VALID; } /// @@ -232,10 +238,15 @@ public unsafe void AddEntry(ReadOnlySpan value, int valueLength, ArgSlice try { - bool canParseID = parseIDString(idSlice, ref id); - if (!canParseID) + var parsedIDStatus = parseIDString(idSlice, ref id); + if (parsedIDStatus == ParsedStreamEntryID.INVALID) + { + writer.WriteError(CmdStrings.RESP_ERR_XADD_INVALID_STREAM_ID); + return; + } + else if (parsedIDStatus == ParsedStreamEntryID.NOT_GREATER) { - writer.WriteError("ERR Syntax"); + writer.WriteError(CmdStrings.RESP_ERR_XADD_ID_NOT_GREATER); return; } From 06a318247458a9194444546eb47b695d82c0b48a Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 13:16:29 -0400 Subject: [PATCH 49/56] added NOMKSTREAM support for XADD --- libs/server/Resp/StreamCommands.cs | 23 ++++++++++++++++++----- libs/server/Stream/Stream.cs | 6 ++++++ libs/server/Stream/StreamManager.cs | 12 ++++++++++-- 3 files changed, 34 insertions(+), 7 deletions(-) diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 60655768acb..460ff4f1ece 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -21,17 +21,27 @@ private unsafe bool StreamAdd(byte respProtocolVersion) return AbortWithErrorMessage(CmdStrings.RESP_ERR_XADD_WRONG_NUM_ARGS); } + int argsParsed = 0; + // Parse the stream key. var key = parseState.GetArgSliceByRef(0); + argsParsed++; + + bool noMkStream = false; + if (argsParsed < parseState.Count && parseState.GetArgSliceByRef(argsParsed).ToString().ToUpper().Equals("NOMKSTREAM")) + { + noMkStream = true; + argsParsed++; + } // Parse the id. We parse as string for easy pattern matching. - var idGiven = parseState.GetArgSliceByRef(1); + var idGiven = parseState.GetArgSliceByRef(argsParsed); // get the number of the remaining key-value pairs - var numPairs = parseState.Count - 2; + var numPairs = parseState.Count - argsParsed; // grab the rest of the input that will mainly be k-v pairs as entry to the stream. - byte* vPtr = parseState.GetArgSliceByRef(2).ptr - sizeof(int); + byte* vPtr = parseState.GetArgSliceByRef(argsParsed).ptr - sizeof(int); int vsize = (int)(recvBufferPtr + endReadHead - vPtr); var streamDataSpan = new ReadOnlySpan(vPtr, vsize); SpanByteAndMemory _output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr)); @@ -51,9 +61,12 @@ private unsafe bool StreamAdd(byte respProtocolVersion) } else { - streamManager.StreamAdd(key, idGiven, streamDataSpan, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream, respProtocolVersion); + streamManager.StreamAdd(key, idGiven, noMkStream, streamDataSpan, vsize, numPairs, ref _output, out byte[] lastStreamKey, out StreamObject lastStream, respProtocolVersion); // since we added to a new stream that was not in the cache, try adding it to the cache - sessionStreamCache.TryAddStreamToCache(lastStreamKey, lastStream); + if (lastStream != null) + { + sessionStreamCache.TryAddStreamToCache(lastStreamKey, lastStream); + } } ProcessOutput(_output); return true; diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index a395d963153..848cb395fd9 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -18,6 +18,12 @@ public enum StreamTrimOpts NONE } + public enum XADDOpts + { + NOMKSTREAM, + NONE + } + public enum ParsedStreamEntryID { VALID, diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 6430a6830dd..86be6fc89de 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -30,6 +30,7 @@ public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) /// /// key/name of the stream /// id of the stream entry + /// if true, do not create a new stream if it does not exist /// payload to the stream /// length of payload to the stream /// # k-v pairs in the payload @@ -37,7 +38,7 @@ public StreamManager(long pageSize, long memorySize, int safeTailRefreshFreqMs) /// key of last stream accessed (for cache) /// reference to last stream accessed (for cache) /// RESP protocol version - public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, ReadOnlySpan value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream, byte respProtocolVersion) + public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, bool noMkStream, ReadOnlySpan value, int valueLength, int numPairs, ref SpanByteAndMemory output, out byte[] streamKey, out StreamObject lastStream, byte respProtocolVersion) { // copy key store this key in the dictionary byte[] key = new byte[keySlice.Length]; @@ -73,7 +74,7 @@ public unsafe void StreamAdd(ArgSlice keySlice, ArgSlice idSlice, ReadOnlySpan Date: Thu, 26 Jun 2025 15:36:30 -0400 Subject: [PATCH 50/56] updated docs --- website/docs/commands/api-compatibility.md | 10 +-- website/docs/commands/data-structures.md | 74 ++++++++++++++++++++++ 2 files changed, 79 insertions(+), 5 deletions(-) diff --git a/website/docs/commands/api-compatibility.md b/website/docs/commands/api-compatibility.md index f6dce718fb3..329198489a3 100644 --- a/website/docs/commands/api-compatibility.md +++ b/website/docs/commands/api-compatibility.md @@ -354,10 +354,10 @@ Note that this list is subject to change as we continue to expand our API comman | | [ZUNION](data-structures.md#zunion) | ➕ | | | | [ZUNIONSTORE](data-structures.md#zunionstore) | ➕ | | | **STREAM** | XACK | ➖ | | -| | XADD | ➖ | | +| | XADD | ➕ | (Does not support Capped Streams) | | | XAUTOCLAIM | ➖ | | | | XCLAIM | ➖ | | -| | XDEL | ➖ | | +| | XDEL | ➕ | | | | XGROUP CREATE | ➖ | | | | XGROUP CREATECONSUMER | ➖ | | | | XGROUP DELCONSUMER | ➖ | | @@ -368,14 +368,14 @@ Note that this list is subject to change as we continue to expand our API comman | | XINFO GROUPS | ➖ | | | | XINFO HELP | ➖ | | | | XINFO STREAM | ➖ | | -| | XLEN | ➖ | | +| | XLEN | ➕ | | | | XPENDING | ➖ | | -| | XRANGE | ➖ | | +| | XRANGE | ➕ | | | | XREAD | ➖ | | | | XREADGROUP | ➖ | | | | XREVRANGE | ➖ | | | | XSETID | ➖ | | -| | XTRIM | ➖ | | +| | XTRIM | ➕ | Does not support near-exact trimming | | **STRING** | [APPEND](raw-string.md#append) | ➕ | | | | [DECR](raw-string.md#decr) | ➕ | | | | [DECRBY](raw-string.md#decrby) | ➕ | | diff --git a/website/docs/commands/data-structures.md b/website/docs/commands/data-structures.md index deb459d6ed4..91bc5ac9199 100644 --- a/website/docs/commands/data-structures.md +++ b/website/docs/commands/data-structures.md @@ -980,6 +980,80 @@ If **destination** already exists, it is overwritten. --- +## Stream + +### XADD + +#### Syntax + +```bash + XADD key [NOMKSTREAM] <* | id> field value [field value ...] +``` +Appends given stream entry to the stream at specified key. If the key does not exist, it is created when running the command. +Creation of the stream can be disabled with the `NOMKSTREAM` option. + +Every entry in the stream is accompanied by a stream entry ID and consists of field-value pairs that are stored/read in the same order as provided by the user. +While the [XADD](#XADD) can auto-generate a unique ID using the `*` character, it is also possible to specify a user-defined ID specified by two 64-bit numbers separated by a `-` character. +The IDs are guaranteed to be incremental. + +**Capped Streams** are not currently supported. + +--- + +### XLEN + +#### Syntax + +```bash + XLEN key +``` +Returns the number of entries inside the stream specified by `key`. If the stream does not exist, returns 0. + +--- + +### XRANGE + +#### Syntax + +```bash + XRANGE key start end [COUNT count] +``` +Returns stream entries matching a given range of IDs. +`start` and `end` can be special IDs (i.e, `-` and `+`) to specify the minimum possible ID and the maximum possible ID inside a stream respectively. +The IDs provided can also be incomplete (i.e., with only the first part of the ID). +Using the `COUNT` option reduces the number of entries returned. + +--- + +### XDEL + +#### Syntax + +```bash + XDEL key id [id ...] +``` +Removes the specified entries from a stream given by key, and returns the number of entries deleted. +If speficied IDs do not exist, the number of entries returned may be less than the number of IDs provided as they are not counted as deleted. + +--- + +### XTRIM + +#### Syntax + +```bash + XTRIM key threshold +``` +Trims the stream by evicting older entries using two strategies: + +- MAXLEN: evicts entries as long as stream's length exceeds specified threshold. +- MINID: evicts entries with IDs lower than threshold where `threshold` is an entry ID. + +Nearly exact trimming using `~` and `LIMIT` clause are not currently supported. +`MINID` defaults to exact trimming, meaning all entries having IDs lower than threshold will be deleted. + +--- + ## Sorted Set ### ZADD From 4b0f911d1243c4fc7e8f5a0ed100fdb80cc5f470 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 15:42:00 -0400 Subject: [PATCH 51/56] minor cleanup + refactor variable name --- libs/server/BTreeIndex/BTreeTrim.cs | 22 +++++++++++----------- libs/server/Resp/StreamCommands.cs | 8 ++++---- libs/server/Stream/Stream.cs | 19 +++++++++++++------ 3 files changed, 28 insertions(+), 21 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 45115af1368..16873085812 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -8,10 +8,10 @@ namespace Garnet.server.BTreeIndex { public unsafe partial class BTree { - public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRemoved, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByID(byte* key, out int underflowingNodes, out ulong entriesTrimmed, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { underflowingNodes = 0; - validKeysRemoved = 0; + entriesTrimmed = 0; numLeavesDeleted = 0; var nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; @@ -28,7 +28,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe { leaf->SetValueValid(i, false); leaf->info->validCount--; - validKeysRemoved++; + entriesTrimmed++; } if (leaf == head) @@ -62,7 +62,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe stats.numLeafNodes--; stats.numKeys -= count; stats.numValidKeys -= validCount; - validKeysRemoved += validCount; + entriesTrimmed += validCount; // deallocate the node Deallocate(ref node); @@ -150,14 +150,14 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong validKeysRe } } - public void TrimByLength(ref BTreeNode* node, ulong length, out ulong validKeysRemoved, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ref BTreeNode* node, ulong length, out ulong entriesTrimmed, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { var depth = stats.depth - 1; ulong currentValidCount = 0; var current = node; int[] internalSlots = new int[MAX_TREE_DEPTH]; int underflowingNodes = 0; - validKeysRemoved = 0; + entriesTrimmed = 0; numLeavesDeleted = 0; headValidKey = default; BTreeNode*[] nodesTraversed = new BTreeNode*[MAX_TREE_DEPTH]; @@ -217,7 +217,7 @@ public void TrimByLength(ref BTreeNode* node, ulong length, out ulong validKeysR stats.numLeafNodes--; stats.numKeys -= count; stats.numValidKeys -= validCount; - validKeysRemoved += validCount; + entriesTrimmed += validCount; // deallocate the node Deallocate(ref leaf); @@ -297,16 +297,16 @@ public void TrimByLength(ref BTreeNode* node, ulong length, out ulong validKeysR } } } - public void TrimByID(byte* key, out ulong validKeysRemoved, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByID(byte* key, out ulong entriesTrimmed, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { int underflowingNodes; - TrimByID(key, out underflowingNodes, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + TrimByID(key, out underflowingNodes, out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted); } - public void TrimByLength(ulong length, out ulong validKeysRemoved, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ulong length, out ulong entriesTrimmed, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) { - TrimByLength(ref root, length, out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + TrimByLength(ref root, length, out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted); } } } \ No newline at end of file diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 460ff4f1ece..95ce61bd149 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -251,7 +251,7 @@ public bool StreamTrim() var trimType = parseState.GetArgSliceByRef(1).ToString().ToUpper(); var trimArg = parseState.GetArgSliceByRef(2); - ulong validKeysRemoved = 0; + ulong entriesTrimmed = 0; StreamTrimOpts optType = StreamTrimOpts.NONE; switch (trimType) { @@ -274,17 +274,17 @@ public bool StreamTrim() bool result; if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { - result = cachedStream.Trim(trimArg, optType, out validKeysRemoved); + result = cachedStream.Trim(trimArg, optType, out entriesTrimmed); } else { - result = streamManager.StreamTrim(key, trimArg, optType, out validKeysRemoved); + result = streamManager.StreamTrim(key, trimArg, optType, out entriesTrimmed); } if (!result) { return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_SYNTAX_ERROR); } - while (!RespWriteUtils.TryWriteInt64((long)validKeysRemoved, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteInt64((long)entriesTrimmed, ref dcurr, dend)) SendAndReset(); return true; } diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index 848cb395fd9..b8b62f2a491 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -515,7 +515,14 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM } } - public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong validKeysRemoved) + /// + /// Trims the stream based on the specified options. + /// + /// length or ID specifying the threshold + /// MAXLEN or MINID + /// number of keys trimmed + /// + public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong entriesTrimmed) { uint numLeavesDeleted = 0; Value headValue = default; @@ -527,21 +534,21 @@ public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong vali case StreamTrimOpts.MAXLEN: if (!RespReadUtils.ReadUlong(out ulong maxLen, ref trimArg.ptr, trimArg.ptr + trimArg.length)) { - validKeysRemoved = 0; + entriesTrimmed = 0; return false; } - index.TrimByLength(maxLen, out validKeysRemoved, out headValue, out var headValidKey, out numLeavesDeleted); + index.TrimByLength(maxLen, out entriesTrimmed, out headValue, out var headValidKey, out numLeavesDeleted); break; case StreamTrimOpts.MINID: if (!parseCompleteID(trimArg, out StreamID minID)) { - validKeysRemoved = 0; + entriesTrimmed = 0; return false; } - index.TrimByID((byte*)Unsafe.AsPointer(ref minID.idBytes[0]), out validKeysRemoved, out headValue, out headValidKey, out numLeavesDeleted); + index.TrimByID((byte*)Unsafe.AsPointer(ref minID.idBytes[0]), out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted); break; default: - validKeysRemoved = 0; + entriesTrimmed = 0; break; } From c121a739b4a5254c7cab38160d9bf116168d4471 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 16:09:12 -0400 Subject: [PATCH 52/56] minor edit --- libs/server/Resp/RespServerSession.cs | 3 --- 1 file changed, 3 deletions(-) diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs index 2aa436135b7..efd3262c507 100644 --- a/libs/server/Resp/RespServerSession.cs +++ b/libs/server/Resp/RespServerSession.cs @@ -204,9 +204,6 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase // Threshold for slow log in ticks (0 means disabled) readonly long slowLogThreshold; - /// - /// Stream cache for the session - /// internal readonly SessionStreamCache sessionStreamCache; public RespServerSession( From fff8af08f8d4eb241499754cf90de9c32dad945f Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Thu, 26 Jun 2025 16:22:54 -0400 Subject: [PATCH 53/56] minor bug fix - enableStreams for ACL tests in server setup --- test/Garnet.test/Resp/ACL/RespCommandTests.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index 1500323db94..058264422e1 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -32,7 +32,7 @@ public class RespCommandTests public void Setup() { TestUtils.DeleteDirectory(TestUtils.MethodTestDir, wait: true); - server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, defaultPassword: DefaultPassword, useAcl: true, enableLua: true); + server = TestUtils.CreateGarnetServer(TestUtils.MethodTestDir, defaultPassword: DefaultPassword, useAcl: true, enableLua: true, enableStreams: true); // Register custom commands so we can test ACL'ing them ClassicAssert.IsTrue(TestUtils.TryGetCustomCommandsInfo(out respCustomCommandsInfo)); From 2796bdaff6b54ad14fe2381196bb9acea39c8089 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Tue, 8 Jul 2025 11:10:51 -0400 Subject: [PATCH 54/56] support exact and approximate trimming of index --- libs/server/BTreeIndex/BTreeInternals.cs | 5 +++ libs/server/BTreeIndex/BTreeTrim.cs | 44 +++++++++++++++++++++--- 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/libs/server/BTreeIndex/BTreeInternals.cs b/libs/server/BTreeIndex/BTreeInternals.cs index 4b79af5a114..2e55cd968d4 100644 --- a/libs/server/BTreeIndex/BTreeInternals.cs +++ b/libs/server/BTreeIndex/BTreeInternals.cs @@ -181,6 +181,11 @@ public void SetValueValid(int index, bool valid) data.values[index].Valid = valid; } + public bool IsValueValid(int index) + { + return data.values[index].Valid; + } + public void InsertTombstone(int index) { data.values[index].Valid = false; diff --git a/libs/server/BTreeIndex/BTreeTrim.cs b/libs/server/BTreeIndex/BTreeTrim.cs index 16873085812..72a635992e7 100644 --- a/libs/server/BTreeIndex/BTreeTrim.cs +++ b/libs/server/BTreeIndex/BTreeTrim.cs @@ -150,7 +150,7 @@ public void TrimByID(byte* key, out int underflowingNodes, out ulong entriesTrim } } - public void TrimByLength(ref BTreeNode* node, ulong length, out ulong entriesTrimmed, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ref BTreeNode* node, ulong length, out ulong entriesTrimmed, out Value headValidValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted, bool approximateTrimming) { var depth = stats.depth - 1; ulong currentValidCount = 0; @@ -193,8 +193,42 @@ public void TrimByLength(ref BTreeNode* node, ulong length, out ulong entriesTri depth--; } - headValidValue = current->GetValue(0); - headValidKey = new ReadOnlySpan(current->GetKey(0), BTreeNode.KEY_SIZE); + if (approximateTrimming) + { + headValidValue = current->GetValue(0); + headValidKey = new ReadOnlySpan(current->GetKey(0), BTreeNode.KEY_SIZE); + } + else + { + ulong keepInCurrent = length - currentValidCount; + ulong kept = 0; + headValidValue = default; + headValidKey = default; + for (int i = 0; i < current->info->count; i++) + { + if (current->IsValueValid(i)) + { + if (kept < keepInCurrent) + { + // Keep this key + if (kept == 0) + { + headValidValue = current->GetValue(i); + headValidKey = new ReadOnlySpan(current->GetKey(i), BTreeNode.KEY_SIZE); + } + kept++; + } + else + { + // Mark as deleted + current->SetValueValid(i, false); + current->info->validCount--; + entriesTrimmed++; + stats.numValidKeys--; + } + } + } + } var leaf = current->info->previous; uint deletedValidCount = 0; @@ -303,10 +337,10 @@ public void TrimByID(byte* key, out ulong entriesTrimmed, out Value headValue, o TrimByID(key, out underflowingNodes, out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted); } - public void TrimByLength(ulong length, out ulong entriesTrimmed, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted) + public void TrimByLength(ulong length, out ulong entriesTrimmed, out Value headValue, out ReadOnlySpan headValidKey, out uint numLeavesDeleted, bool approximateTrimming = false) { - TrimByLength(ref root, length, out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted); + TrimByLength(ref root, length, out entriesTrimmed, out headValue, out headValidKey, out numLeavesDeleted, approximateTrimming); } } } \ No newline at end of file From a2e6c1eeff538a3f201dcf33cbfb6620186fb351 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Tue, 8 Jul 2025 12:29:42 -0400 Subject: [PATCH 55/56] added support for approximate trimming --- libs/server/Resp/StreamCommands.cs | 14 +++++++++++--- libs/server/Stream/Stream.cs | 4 ++-- libs/server/Stream/StreamManager.cs | 4 ++-- test/Garnet.test/Resp/ACL/RespCommandTests.cs | 1 + 4 files changed, 16 insertions(+), 7 deletions(-) diff --git a/libs/server/Resp/StreamCommands.cs b/libs/server/Resp/StreamCommands.cs index 95ce61bd149..35df8615a8c 100644 --- a/libs/server/Resp/StreamCommands.cs +++ b/libs/server/Resp/StreamCommands.cs @@ -249,7 +249,15 @@ public bool StreamTrim() var key = parseState.GetArgSliceByRef(0); var trimType = parseState.GetArgSliceByRef(1).ToString().ToUpper(); - var trimArg = parseState.GetArgSliceByRef(2); + bool approximate = false; + int trimArgIndex = 2; + // Check for optional ~ + if (parseState.Count > 3 && parseState.GetArgSliceByRef(2).ToString() == "~") + { + approximate = true; + trimArgIndex++; + } + var trimArg = parseState.GetArgSliceByRef(trimArgIndex); ulong entriesTrimmed = 0; StreamTrimOpts optType = StreamTrimOpts.NONE; @@ -274,11 +282,11 @@ public bool StreamTrim() bool result; if (sessionStreamCache.TryGetStreamFromCache(key.Span, out StreamObject cachedStream)) { - result = cachedStream.Trim(trimArg, optType, out entriesTrimmed); + result = cachedStream.Trim(trimArg, optType, out entriesTrimmed, approximate); } else { - result = streamManager.StreamTrim(key, trimArg, optType, out entriesTrimmed); + result = streamManager.StreamTrim(key, trimArg, optType, out entriesTrimmed, approximate); } if (!result) { diff --git a/libs/server/Stream/Stream.cs b/libs/server/Stream/Stream.cs index b8b62f2a491..6b74ca6ccee 100644 --- a/libs/server/Stream/Stream.cs +++ b/libs/server/Stream/Stream.cs @@ -522,7 +522,7 @@ public unsafe void ReadRange(string min, string max, int limit, ref SpanByteAndM /// MAXLEN or MINID /// number of keys trimmed /// - public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong entriesTrimmed) + public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong entriesTrimmed, bool approximate = false) { uint numLeavesDeleted = 0; Value headValue = default; @@ -537,7 +537,7 @@ public unsafe bool Trim(ArgSlice trimArg, StreamTrimOpts optType, out ulong entr entriesTrimmed = 0; return false; } - index.TrimByLength(maxLen, out entriesTrimmed, out headValue, out var headValidKey, out numLeavesDeleted); + index.TrimByLength(maxLen, out entriesTrimmed, out headValue, out var headValidKey, out numLeavesDeleted, approximate); break; case StreamTrimOpts.MINID: if (!parseCompleteID(trimArg, out StreamID minID)) diff --git a/libs/server/Stream/StreamManager.cs b/libs/server/Stream/StreamManager.cs index 86be6fc89de..c2168e1d8dd 100644 --- a/libs/server/Stream/StreamManager.cs +++ b/libs/server/Stream/StreamManager.cs @@ -178,7 +178,7 @@ public bool StreamDelete(ArgSlice keySlice, ArgSlice idSlice, out StreamObject l return false; } - public bool StreamTrim(ArgSlice keySlice, ArgSlice trimArg, StreamTrimOpts optType, out ulong validKeysRemoved) + public bool StreamTrim(ArgSlice keySlice, ArgSlice trimArg, StreamTrimOpts optType, out ulong validKeysRemoved, bool approximate = false) { bool foundStream; var key = keySlice.ToArray(); @@ -190,7 +190,7 @@ public bool StreamTrim(ArgSlice keySlice, ArgSlice trimArg, StreamTrimOpts optTy if (foundStream) { - return stream.Trim(trimArg, optType, out validKeysRemoved); + return stream.Trim(trimArg, optType, out validKeysRemoved, approximate); } } return true; // no keys removed so return true diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index 058264422e1..23ebb69b81c 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -6496,6 +6496,7 @@ async Task DoXDelAsync(GarnetClient client) } } + [Test] public async Task XTRIMACLsAsync() { await CheckCommandsAsync( From 0265078f84b07e390209bf3275f7072cbe3c2122 Mon Sep 17 00:00:00 2001 From: Aneesh Raman Date: Tue, 8 Jul 2025 16:57:45 -0400 Subject: [PATCH 56/56] minor doc update --- website/docs/commands/data-structures.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/docs/commands/data-structures.md b/website/docs/commands/data-structures.md index 91bc5ac9199..5ce028ee55c 100644 --- a/website/docs/commands/data-structures.md +++ b/website/docs/commands/data-structures.md @@ -1049,7 +1049,7 @@ Trims the stream by evicting older entries using two strategies: - MAXLEN: evicts entries as long as stream's length exceeds specified threshold. - MINID: evicts entries with IDs lower than threshold where `threshold` is an entry ID. -Nearly exact trimming using `~` and `LIMIT` clause are not currently supported. +`LIMIT` clause is not currently supported. `MINID` defaults to exact trimming, meaning all entries having IDs lower than threshold will be deleted. ---