diff --git a/Directory.Packages.props b/Directory.Packages.props
index b761ccef354..928d36a3852 100644
--- a/Directory.Packages.props
+++ b/Directory.Packages.props
@@ -30,5 +30,6 @@
+
\ No newline at end of file
diff --git a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs
index 7662b533f83..9ac7428ef40 100644
--- a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs
+++ b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs
@@ -25,6 +25,7 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo
static ReadOnlySpan MAIN_STORE => "SSTORE"u8;
static ReadOnlySpan OBJECT_STORE => "OSTORE"u8;
+ static ReadOnlySpan VECTOR_STORE => "VSTORE"u8;
static ReadOnlySpan T => "T"u8;
static ReadOnlySpan F => "F"u8;
@@ -170,14 +171,30 @@ public Task SetSlotRange(Memory state, string nodeid, List<(int, i
///
///
///
- public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMainStore)
+ public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMainStore, bool isVectorSets)
{
currTcsIterationTask = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
tcsQueue.Enqueue(currTcsIterationTask);
curr = offset;
this.isMainStore = isMainStore;
this.ist = IncrementalSendType.MIGRATE;
- var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE;
+ ReadOnlySpan storeType;
+ if (isMainStore)
+ {
+ if (isVectorSets)
+ {
+ storeType = VECTOR_STORE;
+ }
+ else
+ {
+ storeType = MAIN_STORE;
+ }
+ }
+ else
+ {
+ storeType = OBJECT_STORE;
+ }
+
var replaceOption = replace ? T : F;
var arraySize = 6;
@@ -249,7 +266,7 @@ public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMa
///
public Task CompleteMigrate(string sourceNodeId, bool replace, bool isMainStore)
{
- SetClusterMigrateHeader(sourceNodeId, replace, isMainStore);
+ SetClusterMigrateHeader(sourceNodeId, replace, isMainStore, isVectorSets: false);
Debug.Assert(end - curr >= 2);
*curr++ = (byte)'\r';
diff --git a/libs/cluster/Server/ClusterManager.cs b/libs/cluster/Server/ClusterManager.cs
index 2cfbcaf5f07..1dbef4dbed2 100644
--- a/libs/cluster/Server/ClusterManager.cs
+++ b/libs/cluster/Server/ClusterManager.cs
@@ -239,22 +239,27 @@ public string GetInfo()
public static string GetRange(int[] slots)
{
var range = "> ";
- var start = slots[0];
- var end = slots[0];
- for (var i = 1; i < slots.Length + 1; i++)
+ if (slots.Length >= 1)
{
- if (i < slots.Length && slots[i] == end + 1)
- end = slots[i];
- else
+
+ var start = slots[0];
+ var end = slots[0];
+ for (var i = 1; i < slots.Length + 1; i++)
{
- range += $"{start}-{end} ";
- if (i < slots.Length)
- {
- start = slots[i];
+ if (i < slots.Length && slots[i] == end + 1)
end = slots[i];
+ else
+ {
+ range += $"{start}-{end} ";
+ if (i < slots.Length)
+ {
+ start = slots[i];
+ end = slots[i];
+ }
}
}
}
+
return range;
}
diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs
index a35e474a263..0ef36402b84 100644
--- a/libs/cluster/Server/ClusterManagerSlotState.cs
+++ b/libs/cluster/Server/ClusterManagerSlotState.cs
@@ -17,7 +17,10 @@ namespace Garnet.cluster
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
///
/// Cluster manager
diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs
index 51ac87401f0..9af5cf6a02e 100644
--- a/libs/cluster/Server/ClusterProvider.cs
+++ b/libs/cluster/Server/ClusterProvider.cs
@@ -15,12 +15,21 @@
namespace Garnet.cluster
{
+ using BasicContext = BasicContext,
+ SpanByteAllocator>>;
+
using BasicGarnetApi = GarnetApi,
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
+
+ using VectorContext = BasicContext, SpanByteAllocator>>;
///
/// Cluster provider
@@ -100,8 +109,8 @@ public void Start()
}
///
- public IClusterSession CreateClusterSession(TransactionManager txnManager, IGarnetAuthenticator authenticator, UserHandle userHandle, GarnetSessionMetrics garnetSessionMetrics, BasicGarnetApi basicGarnetApi, INetworkSender networkSender, ILogger logger = null)
- => new ClusterSession(this, txnManager, authenticator, userHandle, garnetSessionMetrics, basicGarnetApi, networkSender, logger);
+ public IClusterSession CreateClusterSession(TransactionManager txnManager, IGarnetAuthenticator authenticator, UserHandle userHandle, GarnetSessionMetrics garnetSessionMetrics, BasicGarnetApi basicGarnetApi, BasicContext basicContext, VectorContext vectorContext, INetworkSender networkSender, ILogger logger = null)
+ => new ClusterSession(this, txnManager, authenticator, userHandle, garnetSessionMetrics, basicGarnetApi, basicContext, vectorContext, networkSender, logger);
///
public void UpdateClusterAuth(string clusterUsername, string clusterPassword)
diff --git a/libs/cluster/Server/Migration/MigrateOperation.cs b/libs/cluster/Server/Migration/MigrateOperation.cs
index d4f069a8189..3f677c959ee 100644
--- a/libs/cluster/Server/Migration/MigrateOperation.cs
+++ b/libs/cluster/Server/Migration/MigrateOperation.cs
@@ -2,9 +2,11 @@
// Licensed under the MIT license.
using System;
+using System.Collections.Concurrent;
using System.Collections.Generic;
using Garnet.client;
using Garnet.server;
+using Microsoft.Extensions.Logging;
using Tsavorite.core;
namespace Garnet.cluster
@@ -18,16 +20,25 @@ internal sealed partial class MigrateOperation
public MainStoreScan mss;
public ObjectStoreScan oss;
+ private readonly ConcurrentDictionary vectorSetsIndexKeysToMigrate;
+
readonly MigrateSession session;
readonly GarnetClientSession gcs;
readonly LocalServerSession localServerSession;
public GarnetClientSession Client => gcs;
+ public IEnumerable> VectorSets => vectorSetsIndexKeysToMigrate;
+
public void ThrowIfCancelled() => session._cts.Token.ThrowIfCancellationRequested();
public bool Contains(int slot) => session._sslots.Contains(slot);
+ public bool ContainsNamespace(ulong ns) => session._namespaces?.Contains(ns) ?? false;
+
+ public void EncounteredVectorSet(byte[] key, byte[] value)
+ => vectorSetsIndexKeysToMigrate.TryAdd(key, value);
+
public MigrateOperation(MigrateSession session, Sketch sketch = null, int batchSize = 1 << 18)
{
this.session = session;
@@ -37,6 +48,7 @@ public MigrateOperation(MigrateSession session, Sketch sketch = null, int batchS
mss = new MainStoreScan(this);
oss = new ObjectStoreScan(this);
keysToDelete = [];
+ vectorSetsIndexKeysToMigrate = new(ByteArrayComparer.Instance);
}
public bool Initialize()
@@ -72,7 +84,7 @@ public void Scan(StoreType storeType, ref long currentAddress, long endAddress)
///
///
///
- public bool TrasmitSlots(StoreType storeType)
+ public bool TransmitSlots(StoreType storeType)
{
var bufferSize = 1 << 10;
SectorAlignedMemory buffer = new(bufferSize, 1);
@@ -87,7 +99,7 @@ public bool TrasmitSlots(StoreType storeType)
{
foreach (var key in sketch.argSliceVector)
{
- var spanByte = key.SpanByte;
+ var spanByte = key;
if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, ref spanByte, ref input, ref o, out _))
return false;
@@ -117,7 +129,10 @@ public bool TrasmitSlots(StoreType storeType)
return true;
}
- public bool TransmitKeys(StoreType storeType)
+ ///
+ /// Move keys in sketch out of the given store, UNLESS they are also in .
+ ///
+ public bool TransmitKeys(StoreType storeType, Dictionary vectorSetKeysToIgnore)
{
var bufferSize = 1 << 10;
SectorAlignedMemory buffer = new(bufferSize, 1);
@@ -131,12 +146,30 @@ public bool TransmitKeys(StoreType storeType)
var keys = sketch.Keys;
if (storeType == StoreType.Main)
{
+#if NET9_0_OR_GREATER
+ var ignoreLookup = vectorSetKeysToIgnore.GetAlternateLookup>();
+#endif
+
for (var i = 0; i < keys.Count; i++)
{
if (keys[i].Item2)
continue;
var spanByte = keys[i].Item1.SpanByte;
+
+ // Don't transmit if a Vector Set
+ var isVectorSet =
+ vectorSetKeysToIgnore.Count > 0 &&
+#if NET9_0_OR_GREATER
+ ignoreLookup.ContainsKey(spanByte.AsReadOnlySpan());
+#else
+ vectorSetKeysToIgnore.ContainsKey(spanByte.ToByteArray());
+#endif
+ if (isVectorSet)
+ {
+ continue;
+ }
+
if (!session.WriteOrSendMainStoreKeyValuePair(gcs, localServerSession, ref spanByte, ref input, ref o, out var status))
return false;
@@ -158,8 +191,8 @@ public bool TransmitKeys(StoreType storeType)
if (keys[i].Item2)
continue;
- var argSlice = keys[i].Item1;
- if (!session.WriteOrSendObjectStoreKeyValuePair(gcs, localServerSession, ref argSlice, out var status))
+ var spanByte = keys[i].Item1.SpanByte;
+ if (!session.WriteOrSendObjectStoreKeyValuePair(gcs, localServerSession, ref spanByte, out var status))
return false;
// Skip if key NOTFOUND
@@ -182,6 +215,54 @@ public bool TransmitKeys(StoreType storeType)
return true;
}
+ ///
+ /// Transmit data in namespaces during a MIGRATE ... KEYS operation.
+ ///
+ /// Doesn't delete anything, just scans and transmits.
+ ///
+ public bool TransmitKeysNamespaces(ILogger logger)
+ {
+ var migrateOperation = this;
+
+ if (!migrateOperation.Initialize())
+ return false;
+
+ var workerStartAddress = migrateOperation.session.clusterProvider.storeWrapper.store.Log.BeginAddress;
+ var workerEndAddress = migrateOperation.session.clusterProvider.storeWrapper.store.Log.TailAddress;
+
+ var cursor = workerStartAddress;
+ logger?.LogWarning(" migrate keys (namespaces) scan range [{workerStartAddress}, {workerEndAddress}]", workerStartAddress, workerEndAddress);
+ while (true)
+ {
+ var current = cursor;
+ // Build Sketch
+ migrateOperation.sketch.SetStatus(SketchStatus.INITIALIZING);
+ migrateOperation.Scan(StoreType.Main, ref current, workerEndAddress);
+
+ // Stop if no keys have been found
+ if (migrateOperation.sketch.argSliceVector.IsEmpty) break;
+
+ logger?.LogWarning("Scan from {cursor} to {current} and discovered {count} keys", cursor, current, migrateOperation.sketch.argSliceVector.Count);
+
+ // Transition EPSM to MIGRATING
+ migrateOperation.sketch.SetStatus(SketchStatus.TRANSMITTING);
+ migrateOperation.session.WaitForConfigPropagation();
+
+ // Transmit all keys gathered
+ migrateOperation.TransmitSlots(StoreType.Main);
+
+ // Transition EPSM to DELETING
+ migrateOperation.sketch.SetStatus(SketchStatus.DELETING);
+ migrateOperation.session.WaitForConfigPropagation();
+
+ // Clear keys from buffer
+ migrateOperation.sketch.Clear();
+ cursor = current;
+ }
+
+ return true;
+ }
+
///
/// Delete keys after migration if copyOption is not set
///
@@ -193,7 +274,13 @@ public void DeleteKeys()
{
foreach (var key in sketch.argSliceVector)
{
- var spanByte = key.SpanByte;
+ if (key.MetadataSize == 1)
+ {
+ // Namespace'd keys are not deleted here, but when migration finishes
+ continue;
+ }
+
+ var spanByte = key;
_ = localServerSession.BasicGarnetApi.DELETE(ref spanByte);
}
}
@@ -209,6 +296,19 @@ public void DeleteKeys()
}
}
}
+
+ ///
+ /// Delete a Vector Set after migration if _copyOption is not set.
+ ///
+ public void DeleteVectorSet(ref SpanByte key)
+ {
+ if (session._copyOption)
+ return;
+
+ var delRes = localServerSession.BasicGarnetApi.DELETE(ref key);
+
+ session.logger?.LogDebug("Deleting Vector Set {key} after migration: {delRes}", System.Text.Encoding.UTF8.GetString(key.AsReadOnlySpan()), delRes);
+ }
}
}
}
\ No newline at end of file
diff --git a/libs/cluster/Server/Migration/MigrateScanFunctions.cs b/libs/cluster/Server/Migration/MigrateScanFunctions.cs
index 03cb23d1af8..25d9f5da3d3 100644
--- a/libs/cluster/Server/Migration/MigrateScanFunctions.cs
+++ b/libs/cluster/Server/Migration/MigrateScanFunctions.cs
@@ -36,10 +36,34 @@ public unsafe bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMeta
if (ClusterSession.Expired(ref value))
return true;
- var s = HashSlotUtils.HashSlot(ref key);
- // Check if key belongs to slot that is being migrated and if it can be added to our buffer
- if (mss.Contains(s) && !mss.sketch.TryHashAndStore(key.AsSpan()))
- return false;
+ // TODO: Some other way to detect namespaces
+ if (key.MetadataSize == 1)
+ {
+ var ns = key.GetNamespaceInPayload();
+
+ if (mss.ContainsNamespace(ns) && !mss.sketch.TryHashAndStore(ns, key.AsSpan()))
+ return false;
+ }
+ else
+ {
+ var s = HashSlotUtils.HashSlot(ref key);
+
+ // Check if key belongs to slot that is being migrated...
+ if (mss.Contains(s))
+ {
+ if (recordMetadata.RecordInfo.VectorSet)
+ {
+ // We can't delete the vector set _yet_ nor can we migrate it,
+ // we just need to remember it to migrate once the associated namespaces are all moved over
+ mss.EncounteredVectorSet(key.ToByteArray(), value.ToByteArray());
+ }
+ else if (!mss.sketch.TryHashAndStore(key.AsSpan()))
+ {
+ // Out of space, end scan for now
+ return false;
+ }
+ }
+ }
return true;
}
diff --git a/libs/cluster/Server/Migration/MigrateSession.cs b/libs/cluster/Server/Migration/MigrateSession.cs
index 16c4cb481dd..cd59a66d347 100644
--- a/libs/cluster/Server/Migration/MigrateSession.cs
+++ b/libs/cluster/Server/Migration/MigrateSession.cs
@@ -2,6 +2,7 @@
// Licensed under the MIT license.
using System;
+using System.Collections.Frozen;
using System.Collections.Generic;
using System.Linq;
using System.Net;
@@ -48,6 +49,9 @@ internal sealed unsafe partial class MigrateSession : IDisposable
readonly HashSet _sslots;
readonly CancellationTokenSource _cts = new();
+ HashSet _namespaces;
+ FrozenDictionary _namespaceMap;
+
///
/// Get endpoint of target node
///
@@ -276,9 +280,10 @@ public bool TrySetSlotRanges(string nodeid, MigrateState state)
Status = MigrateState.FAIL;
return false;
}
- logger?.LogTrace("[Completed] SETSLOT {slots} {state} {nodeid}", ClusterManager.GetRange([.. _sslots]), state, nodeid == null ? "" : nodeid);
+ logger?.LogTrace("[Completed] SETSLOT {slots} {state} {nodeid}", ClusterManager.GetRange([.. _sslots]), state, nodeid ?? "");
return true;
- }, TaskContinuationOptions.OnlyOnRanToCompletion).WaitAsync(_timeout, _cts.Token).Result;
+ }, TaskContinuationOptions.OnlyOnRanToCompletion)
+ .WaitAsync(_timeout, _cts.Token).Result;
}
catch (Exception ex)
{
@@ -338,6 +343,8 @@ public bool TryRecoverFromFailure()
// This will execute the equivalent of SETSLOTRANGE STABLE for the slots of the failed migration task
ResetLocalSlot();
+ // TODO: Need to relinquish any migrating Vector Set contexts from target node
+
// Log explicit migration failure.
Status = MigrateState.FAIL;
return true;
diff --git a/libs/cluster/Server/Migration/MigrateSessionCommonUtils.cs b/libs/cluster/Server/Migration/MigrateSessionCommonUtils.cs
index 835f755a4b8..a11059bfe49 100644
--- a/libs/cluster/Server/Migration/MigrateSessionCommonUtils.cs
+++ b/libs/cluster/Server/Migration/MigrateSessionCommonUtils.cs
@@ -2,6 +2,7 @@
// Licensed under the MIT license.
using System;
+using System.Diagnostics;
using System.Threading.Tasks;
using Garnet.client;
using Garnet.server;
@@ -29,6 +30,18 @@ private bool WriteOrSendMainStoreKeyValuePair(GarnetClientSession gcs, LocalServ
value = ref SpanByte.ReinterpretWithoutLength(o.Memory.Memory.Span);
}
+ // Map up any namespaces as needed
+ // TODO: Better way to do "has namespace"
+ if (key.MetadataSize == 1)
+ {
+ var oldNs = key.GetNamespaceInPayload();
+ if (_namespaceMap.TryGetValue(oldNs, out var newNs))
+ {
+ Debug.Assert(newNs <= byte.MaxValue, "Namespace too large");
+ key.SetNamespaceInPayload((byte)newNs);
+ }
+ }
+
// Write key to network buffer if it has not expired
if (!ClusterSession.Expired(ref value) && !WriteOrSendMainStoreKeyValuePair(gcs, ref key, ref value))
return false;
@@ -39,7 +52,7 @@ bool WriteOrSendMainStoreKeyValuePair(GarnetClientSession gcs, ref SpanByte key,
{
// Check if we need to initialize cluster migrate command arguments
if (gcs.NeedsInitialization)
- gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true);
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: false);
// Try write serialized key value to client buffer
while (!gcs.TryWriteKeyValueSpanByte(ref key, ref value, out var task))
@@ -49,15 +62,15 @@ bool WriteOrSendMainStoreKeyValuePair(GarnetClientSession gcs, ref SpanByte key,
return false;
// re-initialize cluster migrate command parameters
- gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true);
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: false);
}
return true;
}
}
- private bool WriteOrSendObjectStoreKeyValuePair(GarnetClientSession gcs, LocalServerSession localServerSession, ref ArgSlice key, out GarnetStatus status)
+ private bool WriteOrSendObjectStoreKeyValuePair(GarnetClientSession gcs, LocalServerSession localServerSession, ref SpanByte key, out GarnetStatus status)
{
- var keyByteArray = key.ToArray();
+ var keyByteArray = key.AsReadOnlySpan().ToArray();
ObjectInput input = default;
GarnetObjectStoreOutput value = default;
@@ -81,14 +94,14 @@ bool WriteOrSendObjectStoreKeyValuePair(GarnetClientSession gcs, byte[] key, byt
{
// Check if we need to initialize cluster migrate command arguments
if (gcs.NeedsInitialization)
- gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false);
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false, isVectorSets: false);
while (!gcs.TryWriteKeyValueByteArray(key, value, expiration, out var task))
{
// Flush key value pairs in the buffer
if (!HandleMigrateTaskResponse(task))
return false;
- gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false);
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: false, isVectorSets: false);
}
return true;
}
diff --git a/libs/cluster/Server/Migration/MigrateSessionKeys.cs b/libs/cluster/Server/Migration/MigrateSessionKeys.cs
index 294b4ae3172..a49b5eabf45 100644
--- a/libs/cluster/Server/Migration/MigrateSessionKeys.cs
+++ b/libs/cluster/Server/Migration/MigrateSessionKeys.cs
@@ -2,6 +2,8 @@
// Licensed under the MIT license.
using System;
+using System.Collections.Generic;
+using System.Linq;
using Garnet.server;
using Microsoft.Extensions.Logging;
using Tsavorite.core;
@@ -33,13 +35,78 @@ private bool MigrateKeysFromMainStore()
migrateTask.sketch.SetStatus(SketchStatus.TRANSMITTING);
WaitForConfigPropagation();
+ // Discover Vector Sets linked namespaces
+ var indexesToMigrate = new Dictionary(ByteArrayComparer.Instance);
+ _namespaces = clusterProvider.storeWrapper.DefaultDatabase.VectorManager.GetNamespacesForKeys(clusterProvider.storeWrapper, migrateTask.sketch.Keys.Select(t => t.Item1.ToArray()), indexesToMigrate);
+
+ // If we have any namespaces, that implies Vector Sets, and if we have any of THOSE
+ // we need to reserve destination sets on the other side
+ if ((_namespaces?.Count ?? 0) > 0 && !ReserveDestinationVectorSetsAsync().GetAwaiter().GetResult())
+ {
+ logger?.LogError("Failed to reserve destination vector sets, migration failed");
+ return false;
+ }
+
// Transmit keys from main store
- if (!migrateTask.TransmitKeys(StoreType.Main))
+ if (!migrateTask.TransmitKeys(StoreType.Main, indexesToMigrate))
{
logger?.LogError("Failed transmitting keys from main store");
return false;
}
+ if ((_namespaces?.Count ?? 0) > 0)
+ {
+ // Actually move element data over
+ if (!migrateTask.TransmitKeysNamespaces(logger))
+ {
+ logger?.LogError("Failed to transmit vector set (namespaced) element data, migration failed");
+ return false;
+ }
+
+ // Move the indexes over
+ var gcs = migrateTask.Client;
+
+ foreach (var (key, value) in indexesToMigrate)
+ {
+ // Update the index context as we move it, so it arrives on the destination node pointed at the appropriate
+ // namespaces for element data
+ VectorManager.ReadIndex(value, out var oldContext, out _, out _, out _, out _, out _, out _, out _);
+
+ var newContext = _namespaceMap[oldContext];
+ VectorManager.SetContextForMigration(value, newContext);
+
+ unsafe
+ {
+ fixed (byte* keyPtr = key, valuePtr = value)
+ {
+ var keySpan = SpanByte.FromPinnedPointer(keyPtr, key.Length);
+ var valSpan = SpanByte.FromPinnedPointer(valuePtr, value.Length);
+
+ if (gcs.NeedsInitialization)
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: true);
+
+ while (!gcs.TryWriteKeyValueSpanByte(ref keySpan, ref valSpan, out var task))
+ {
+ if (!HandleMigrateTaskResponse(task))
+ {
+ logger?.LogCritical("Failed to migrate Vector Set key {key} during migration", keySpan);
+ return false;
+ }
+
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: true);
+ }
+ }
+ }
+ }
+
+ if (!HandleMigrateTaskResponse(gcs.SendAndResetIterationBuffer()))
+ {
+ logger?.LogCritical("Final flush after Vector Set migration failed");
+ return false;
+ }
+ }
+
+ // Final cleanup, which will also delete Vector Sets
DeleteKeys();
}
finally
@@ -68,7 +135,7 @@ private bool MigrateKeysFromObjectStore()
WaitForConfigPropagation();
// Transmit keys from object store
- if (!migrateTask.TransmitKeys(StoreType.Object))
+ if (!migrateTask.TransmitKeys(StoreType.Object, new(ByteArrayComparer.Instance)))
{
logger?.LogError("Failed transmitting keys from object store");
return false;
diff --git a/libs/cluster/Server/Migration/MigrateSessionSlots.cs b/libs/cluster/Server/Migration/MigrateSessionSlots.cs
index 0d153cc4aa0..7ce25a4048d 100644
--- a/libs/cluster/Server/Migration/MigrateSessionSlots.cs
+++ b/libs/cluster/Server/Migration/MigrateSessionSlots.cs
@@ -2,17 +2,68 @@
// Licensed under the MIT license.
using System;
+using System.Collections.Frozen;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Linq;
using System.Threading.Tasks;
#if DEBUG
using Garnet.common;
#endif
using Garnet.server;
using Microsoft.Extensions.Logging;
+using Tsavorite.core;
namespace Garnet.cluster
{
internal sealed partial class MigrateSession : IDisposable
{
+ ///
+ /// Attempts to reserve contexts on the destination node for migrating vector sets.
+ ///
+ /// This maps roughly to "for each namespaces, reserve one context, record the mapping".
+ ///
+ public async Task ReserveDestinationVectorSetsAsync()
+ {
+ Debug.Assert((_namespaces.Count % (int)VectorManager.ContextStep) == 0, "Expected to be migrating Vector Sets, and thus to have an even number of namespaces");
+
+ var neededContexts = _namespaces.Count / (int)VectorManager.ContextStep;
+
+ try
+ {
+ var reservedCtxs = await migrateOperation[0].Client.ExecuteForArrayAsync("CLUSTER", "RESERVE", "VECTOR_SET_CONTEXTS", neededContexts.ToString());
+
+ var rootNamespacesMigrating = _namespaces.Where(static x => (x % VectorManager.ContextStep) == 0);
+
+ var nextReservedIx = 0;
+
+ var namespaceMap = new Dictionary();
+
+ foreach (var migratingContext in rootNamespacesMigrating)
+ {
+ var toMapTo = ulong.Parse(reservedCtxs[nextReservedIx]);
+ for (var i = 0U; i < VectorManager.ContextStep; i++)
+ {
+ var fromCtx = migratingContext + i;
+ var toCtx = toMapTo + i;
+
+ namespaceMap[fromCtx] = toCtx;
+ }
+
+ nextReservedIx++;
+ }
+
+ _namespaceMap = namespaceMap.ToFrozenDictionary();
+
+ return true;
+ }
+ catch (Exception ex)
+ {
+ logger?.LogError(ex, "Failed to reserve {count} Vector Set contexts on destination node {node}", neededContexts, _targetNodeId);
+ return false;
+ }
+ }
+
///
/// Migrate Slots inline driver
///
@@ -61,6 +112,60 @@ async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress
try
{
await Task.WhenAll(migrateOperationRunners).WaitAsync(_timeout, _cts.Token).ConfigureAwait(false);
+
+ // Handle migration of discovered Vector Set keys now that they're namespaces have been moved
+ if (storeType == StoreType.Main)
+ {
+ var vectorSets = migrateOperation.SelectMany(static mo => mo.VectorSets).GroupBy(static g => g.Key, ByteArrayComparer.Instance).ToDictionary(static g => g.Key, g => g.First().Value, ByteArrayComparer.Instance);
+
+ if (vectorSets.Count > 0)
+ {
+ var gcs = migrateOperation[0].Client;
+
+ foreach (var (key, value) in vectorSets)
+ {
+ // Update the index context as we move it, so it arrives on the destination node pointed at the appropriate
+ // namespaces for element data
+ VectorManager.ReadIndex(value, out var oldContext, out _, out _, out _, out _, out _, out _, out _);
+
+ var newContext = _namespaceMap[oldContext];
+ VectorManager.SetContextForMigration(value, newContext);
+
+ unsafe
+ {
+ fixed (byte* keyPtr = key, valuePtr = value)
+ {
+ var keySpan = SpanByte.FromPinnedPointer(keyPtr, key.Length);
+ var valSpan = SpanByte.FromPinnedPointer(valuePtr, value.Length);
+
+ if (gcs.NeedsInitialization)
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: true);
+
+ while (!gcs.TryWriteKeyValueSpanByte(ref keySpan, ref valSpan, out var task))
+ {
+ if (!HandleMigrateTaskResponse(task))
+ {
+ logger?.LogCritical("Failed to migrate Vector Set key {key} during migration", keySpan);
+ return false;
+ }
+
+ gcs.SetClusterMigrateHeader(_sourceNodeId, _replaceOption, isMainStore: true, isVectorSets: true);
+ }
+
+ // Force a flush before doing the delete, in case that fails
+ if (!HandleMigrateTaskResponse(gcs.SendAndResetIterationBuffer()))
+ {
+ logger?.LogCritical("Flush failed before deletion of Vector Set {key} duration migration", keySpan);
+ return false;
+ }
+
+ // Delete the index on this node now that it's moved over to the destination node
+ migrateOperation[0].DeleteVectorSet(ref keySpan);
+ }
+ }
+ }
+ }
+ }
}
catch (Exception ex)
{
@@ -68,6 +173,7 @@ async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress
_cts.Cancel();
return false;
}
+
return true;
}
@@ -103,7 +209,7 @@ Task ScanStoreTask(int taskId, StoreType storeType, long beginAddress, lon
WaitForConfigPropagation();
// Transmit all keys gathered
- migrateOperation.TrasmitSlots(storeType);
+ migrateOperation.TransmitSlots(storeType);
// Transition EPSM to DELETING
migrateOperation.sketch.SetStatus(SketchStatus.DELETING);
diff --git a/libs/cluster/Server/Migration/MigrationDriver.cs b/libs/cluster/Server/Migration/MigrationDriver.cs
index d2e6af5c1c2..eeda6d6d7e2 100644
--- a/libs/cluster/Server/Migration/MigrationDriver.cs
+++ b/libs/cluster/Server/Migration/MigrationDriver.cs
@@ -78,6 +78,19 @@ private async Task BeginAsyncMigrationTask()
if (!clusterProvider.BumpAndWaitForEpochTransition()) return;
#endregion
+ // Acquire namespaces at this point, after slots have been switch to migration
+ _namespaces = clusterProvider.storeWrapper.DefaultDatabase.VectorManager.GetNamespacesForHashSlots(_sslots);
+
+ // If we have any namespaces, that implies Vector Sets, and if we have any of THOSE
+ // we need to reserve destination sets on the other side
+ if ((_namespaces?.Count ?? 0) > 0 && !await ReserveDestinationVectorSetsAsync())
+ {
+ logger?.LogError("Failed to reserve destination vector sets, migration failed");
+ TryRecoverFromFailure();
+ Status = MigrateState.FAIL;
+ return;
+ }
+
#region migrateData
// Migrate actual data
if (!await MigrateSlotsDriverInline())
@@ -87,6 +100,7 @@ private async Task BeginAsyncMigrationTask()
Status = MigrateState.FAIL;
return;
}
+
#endregion
#region transferSlotOwnnershipToTargetNode
diff --git a/libs/cluster/Server/Migration/Sketch.cs b/libs/cluster/Server/Migration/Sketch.cs
index 4c1ff3e376e..59f3d0bc4a5 100644
--- a/libs/cluster/Server/Migration/Sketch.cs
+++ b/libs/cluster/Server/Migration/Sketch.cs
@@ -44,6 +44,19 @@ public bool TryHashAndStore(Span key)
return true;
}
+ public bool TryHashAndStore(ulong ns, Span key)
+ {
+ if (!argSliceVector.TryAddItem(ns, key))
+ return false;
+
+ var slot = (int)HashUtils.MurmurHash2x64A(key, seed: (uint)ns) & (size - 1);
+ var byteOffset = slot >> 3;
+ var bitOffset = slot & 7;
+ bitmap[byteOffset] = (byte)(bitmap[byteOffset] | (1UL << bitOffset));
+
+ return true;
+ }
+
///
/// Hash key to bloomfilter and store it for future use (NOTE: Use only with KEYS option)
///
@@ -65,7 +78,19 @@ public unsafe void HashAndStore(ref ArgSlice key)
///
public unsafe bool Probe(SpanByte key, out SketchStatus status)
{
- var slot = (int)HashUtils.MurmurHash2x64A(key.ToPointer(), key.Length) & (size - 1);
+ int slot;
+
+ // TODO: better way to detect namespace
+ if (key.MetadataSize == 1)
+ {
+ var ns = key.GetNamespaceInPayload();
+ slot = (int)HashUtils.MurmurHash2x64A(key.ToPointer(), key.Length, seed: (uint)ns) & (size - 1);
+ }
+ else
+ {
+ slot = (int)HashUtils.MurmurHash2x64A(key.ToPointer(), key.Length) & (size - 1);
+ }
+
var byteOffset = slot >> 3;
var bitOffset = slot & 7;
diff --git a/libs/cluster/Session/ClusterCommands.cs b/libs/cluster/Session/ClusterCommands.cs
index 104e05144b7..d938b710340 100644
--- a/libs/cluster/Session/ClusterCommands.cs
+++ b/libs/cluster/Session/ClusterCommands.cs
@@ -135,7 +135,7 @@ private bool TryParseSlots(int startIdx, out HashSet slots, out ReadOnlySpa
/// Subcommand to execute.
/// True if number of parameters is invalid
/// True if command is fully processed, false if more processing is needed.
- private void ProcessClusterCommands(RespCommand command, out bool invalidParameters)
+ private void ProcessClusterCommands(RespCommand command, VectorManager vectorManager, out bool invalidParameters)
{
_ = command switch
{
@@ -173,6 +173,7 @@ private void ProcessClusterCommands(RespCommand command, out bool invalidParamet
RespCommand.CLUSTER_PUBLISH or RespCommand.CLUSTER_SPUBLISH => NetworkClusterPublish(out invalidParameters),
RespCommand.CLUSTER_REPLICAS => NetworkClusterReplicas(out invalidParameters),
RespCommand.CLUSTER_REPLICATE => NetworkClusterReplicate(out invalidParameters),
+ RespCommand.CLUSTER_RESERVE => NetworkClusterReserve(vectorManager, out invalidParameters),
RespCommand.CLUSTER_RESET => NetworkClusterReset(out invalidParameters),
RespCommand.CLUSTER_SEND_CKPT_FILE_SEGMENT => NetworkClusterSendCheckpointFileSegment(out invalidParameters),
RespCommand.CLUSTER_SEND_CKPT_METADATA => NetworkClusterSendCheckpointMetadata(out invalidParameters),
diff --git a/libs/cluster/Session/ClusterKeyIterationFunctions.cs b/libs/cluster/Session/ClusterKeyIterationFunctions.cs
index 54d91d6cd3d..af011f3798c 100644
--- a/libs/cluster/Session/ClusterKeyIterationFunctions.cs
+++ b/libs/cluster/Session/ClusterKeyIterationFunctions.cs
@@ -34,6 +34,14 @@ internal sealed class MainStoreCountKeys : IScanIteratorFunctions keys, int slot, int maxKeyCount)
public bool SingleReader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords, out CursorRecordResult cursorRecordResult)
{
+ // TODO: better way to detect namespace
+ if (key.MetadataSize == 1)
+ {
+ // Namespace means not visible
+ cursorRecordResult = CursorRecordResult.Skip;
+ return true;
+ }
+
cursorRecordResult = CursorRecordResult.Accept; // default; not used here, out CursorRecordResult cursorRecordResult
+
if (HashSlotUtils.HashSlot(ref key) == slot && !Expired(ref value))
keys.Add(key.ToByteArray());
return keys.Count < maxKeyCount;
diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs
index 45780b2d2bf..bfe1f6c475a 100644
--- a/libs/cluster/Session/ClusterSession.cs
+++ b/libs/cluster/Session/ClusterSession.cs
@@ -12,12 +12,21 @@
namespace Garnet.cluster
{
+ using BasicContext = BasicContext,
+ SpanByteAllocator>>;
+
using BasicGarnetApi = GarnetApi,
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
+
+ using VectorContext = BasicContext, SpanByteAllocator>>;
internal sealed unsafe partial class ClusterSession : IClusterSession
{
@@ -57,7 +66,20 @@ internal sealed unsafe partial class ClusterSession : IClusterSession
///
public IGarnetServer Server { get; set; }
- public ClusterSession(ClusterProvider clusterProvider, TransactionManager txnManager, IGarnetAuthenticator authenticator, UserHandle userHandle, GarnetSessionMetrics sessionMetrics, BasicGarnetApi basicGarnetApi, INetworkSender networkSender, ILogger logger = null)
+ private VectorContext vectorContext;
+ private BasicContext basicContext;
+
+ public ClusterSession(
+ ClusterProvider clusterProvider,
+ TransactionManager txnManager,
+ IGarnetAuthenticator authenticator,
+ UserHandle userHandle,
+ GarnetSessionMetrics sessionMetrics,
+ BasicGarnetApi basicGarnetApi,
+ BasicContext basicContext,
+ VectorContext vectorContext,
+ INetworkSender networkSender,
+ ILogger logger = null)
{
this.clusterProvider = clusterProvider;
this.authenticator = authenticator;
@@ -65,11 +87,13 @@ public ClusterSession(ClusterProvider clusterProvider, TransactionManager txnMan
this.txnManager = txnManager;
this.sessionMetrics = sessionMetrics;
this.basicGarnetApi = basicGarnetApi;
+ this.basicContext = basicContext;
+ this.vectorContext = vectorContext;
this.networkSender = networkSender;
this.logger = logger;
}
- public void ProcessClusterCommands(RespCommand command, ref SessionParseState parseState, ref byte* dcurr, ref byte* dend)
+ public void ProcessClusterCommands(RespCommand command, VectorManager vectorManager, ref SessionParseState parseState, ref byte* dcurr, ref byte* dend)
{
this.dcurr = dcurr;
this.dend = dend;
@@ -89,7 +113,7 @@ public void ProcessClusterCommands(RespCommand command, ref SessionParseState pa
return;
}
- ProcessClusterCommands(command, out invalidParameters);
+ ProcessClusterCommands(command, vectorManager, out invalidParameters);
}
else
{
diff --git a/libs/cluster/Session/MigrateCommand.cs b/libs/cluster/Session/MigrateCommand.cs
index 897ca187e02..f884d4d27b1 100644
--- a/libs/cluster/Session/MigrateCommand.cs
+++ b/libs/cluster/Session/MigrateCommand.cs
@@ -13,7 +13,7 @@ namespace Garnet.cluster
{
internal sealed unsafe partial class ClusterSession : IClusterSession
{
- public static bool Expired(ref SpanByte value) => value.MetadataSize > 0 && value.ExtraMetadata < DateTimeOffset.UtcNow.Ticks;
+ public static bool Expired(ref SpanByte value) => value.MetadataSize == 8 && value.ExtraMetadata < DateTimeOffset.UtcNow.Ticks;
public static bool Expired(ref IGarnetObject value) => value.Expiration != 0 && value.Expiration < DateTimeOffset.UtcNow.Ticks;
diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs
index 3dd58cf82a1..5fe9c8d1c4c 100644
--- a/libs/cluster/Session/RespClusterMigrateCommands.cs
+++ b/libs/cluster/Session/RespClusterMigrateCommands.cs
@@ -17,7 +17,10 @@ namespace Garnet.cluster
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
internal sealed unsafe partial class ClusterSession : IClusterSession
{
@@ -103,18 +106,30 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
continue;
}
- var slot = HashSlotUtils.HashSlot(ref key);
- if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state
+ // TODO: better way to handle namespaces
+ if (key.MetadataSize == 1)
{
- migrateState = 1;
- i++;
- continue;
+ // This is a Vector Set namespace key being migrated - it won't necessarily look like it's "in" a hash slot
+ // because it's dependent on some other key (the index key) being migrated which itself is in a moving hash slot
+
+ clusterProvider.storeWrapper.DefaultDatabase.VectorManager.HandleMigratedElementKey(ref basicContext, ref vectorContext, ref key, ref value);
+ }
+ else
+ {
+ var slot = HashSlotUtils.HashSlot(ref key);
+ if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state
+ {
+ migrateState = 1;
+ i++;
+ continue;
+ }
+
+ // Set if key replace flag is set or key does not exist
+ var keySlice = new ArgSlice(key.ToPointer(), key.Length);
+ if (replaceOption || !Exists(ref keySlice))
+ _ = basicGarnetApi.SET(ref key, ref value);
}
- // Set if key replace flag is set or key does not exist
- var keySlice = new ArgSlice(key.ToPointer(), key.Length);
- if (replaceOption || !Exists(ref keySlice))
- _ = basicGarnetApi.SET(ref key, ref value);
i++;
}
}
@@ -150,6 +165,35 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
i++;
}
}
+ else if (storeTypeSpan.Equals("VSTORE", StringComparison.OrdinalIgnoreCase))
+ {
+ // This is the subset of the main store that holds Vector Set _index_ keys
+ //
+ // Namespace'd element keys are handled by the SSTORE path
+
+ var keyCount = *(int*)payloadPtr;
+ payloadPtr += 4;
+ var i = 0;
+
+ TrackImportProgress(keyCount, isMainStore: true, keyCount == 0);
+ while (i < keyCount)
+ {
+ ref var key = ref SpanByte.Reinterpret(payloadPtr);
+ payloadPtr += key.TotalSize;
+ ref var value = ref SpanByte.Reinterpret(payloadPtr);
+ payloadPtr += value.TotalSize;
+
+ // An error has occurred
+ if (migrateState > 0)
+ {
+ i++;
+ continue;
+ }
+
+ clusterProvider.storeWrapper.DefaultDatabase.VectorManager.HandleMigratedIndexKey(clusterProvider.storeWrapper.DefaultDatabase, clusterProvider.storeWrapper, ref key, ref value);
+ i++;
+ }
+ }
else
{
throw new Exception("CLUSTER MIGRATE STORE TYPE ERROR!");
diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs
index b30a3ff00f4..8bc596c8e54 100644
--- a/libs/cluster/Session/RespClusterReplicationCommands.cs
+++ b/libs/cluster/Session/RespClusterReplicationCommands.cs
@@ -115,6 +115,59 @@ private bool NetworkClusterReplicate(out bool invalidParameters)
return true;
}
+ ///
+ /// Implements CLUSTER reserve command (only for internode use).
+ ///
+ /// Allows for pre-migration reservation of certain resources.
+ ///
+ /// For now, this is only used for Vector Sets.
+ ///
+ private bool NetworkClusterReserve(VectorManager vectorManager, out bool invalidParameters)
+ {
+ if (parseState.Count < 2)
+ {
+ invalidParameters = true;
+ return true;
+ }
+
+ var kind = parseState.GetArgSliceByRef(0);
+ if (!kind.ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("VECTOR_SET_CONTEXTS"u8))
+ {
+ while (!RespWriteUtils.TryWriteError("Unrecognized reservation type"u8, ref dcurr, dend))
+ SendAndReset();
+
+ invalidParameters = false;
+ return true;
+ }
+
+ if (!parseState.TryGetInt(1, out var numVectorSetContexts) || numVectorSetContexts <= 0)
+ {
+ invalidParameters = true;
+ return true;
+ }
+
+ invalidParameters = false;
+
+ if (!vectorManager.TryReserveContextsForMigration(ref vectorContext, numVectorSetContexts, out var newContexts))
+ {
+ while (!RespWriteUtils.TryWriteError("Insufficients contexts available to reserve"u8, ref dcurr, dend))
+ SendAndReset();
+
+ return true;
+ }
+
+ while (!RespWriteUtils.TryWriteArrayLength(newContexts.Count, ref dcurr, dend))
+ SendAndReset();
+
+ foreach (var ctx in newContexts)
+ {
+ while (!RespWriteUtils.TryWriteInt64AsSimpleString((long)ctx, ref dcurr, dend))
+ SendAndReset();
+ }
+
+ return true;
+ }
+
///
/// Implements CLUSTER aofsync command (only for internode use)
///
diff --git a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
index 0416c064d43..b2bdc1fba17 100644
--- a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
+++ b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
@@ -2,6 +2,7 @@
// Licensed under the MIT license.
using System;
+using System.Diagnostics;
using System.Runtime.CompilerServices;
using System.Threading;
using Garnet.server;
@@ -23,9 +24,13 @@ private bool CheckIfKeyExists(byte[] key)
}
}
- private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref ArgSlice keySlice, bool readOnly, byte SessionAsking, int slot = -1)
+ private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref ArgSlice keySlice, bool readOnly, byte SessionAsking, bool isVectorSetWriteCommand, int slot = -1)
{
- return readOnly ? SingleKeyReadSlotVerify(ref config, ref keySlice) : SingleKeyReadWriteSlotVerify(ref config, ref keySlice);
+ Debug.Assert(!isVectorSetWriteCommand || (isVectorSetWriteCommand && !readOnly), "Shouldn't see Vector Set writes and readonly at same time");
+
+ var ret = readOnly ? SingleKeyReadSlotVerify(ref config, ref keySlice) : SingleKeyReadWriteSlotVerify(isVectorSetWriteCommand, ref config, ref keySlice);
+
+ return ret;
[MethodImpl(MethodImplOptions.AggressiveInlining)]
ClusterSlotVerificationResult SingleKeyReadSlotVerify(ref ClusterConfig config, ref ArgSlice keySlice)
@@ -69,12 +74,20 @@ ClusterSlotVerificationResult SingleKeyReadSlotVerify(ref ClusterConfig config,
}
[MethodImpl(MethodImplOptions.AggressiveInlining)]
- ClusterSlotVerificationResult SingleKeyReadWriteSlotVerify(ref ClusterConfig config, ref ArgSlice keySlice)
+ ClusterSlotVerificationResult SingleKeyReadWriteSlotVerify(bool isVectorSetWriteCommand, ref ClusterConfig config, ref ArgSlice keySlice)
{
var _slot = slot == -1 ? ArgSliceUtils.HashSlot(ref keySlice) : (ushort)slot;
+
+ tryAgain:
var IsLocal = config.IsLocal(_slot, readWriteSession: readWriteSession);
var state = config.GetState(_slot);
+ if (isVectorSetWriteCommand && state is SlotState.IMPORTING or SlotState.MIGRATING)
+ {
+ WaitForSlotToStabalize(_slot, ref keySlice, ref config);
+ goto tryAgain;
+ }
+
// Redirect r/w requests towards primary
if (config.LocalNodeRole == NodeRole.REPLICA && !readWriteSession)
return new(SlotVerifiedState.MOVED, _slot);
@@ -123,18 +136,35 @@ bool CanOperateOnKey(ref ArgSlice key, int slot, bool readOnly)
}
return Exists(ref key);
}
+
+ void WaitForSlotToStabalize(ushort slot, ref ArgSlice keySlice, ref ClusterConfig config)
+ {
+ // For Vector Set ops specifically, we need a slot to be stable (or faulted, but not migrating) before writes can proceed
+ //
+ // This isn't key specific because we can't know the Vector Sets being migrated in advance, only that the slot is moving
+
+ do
+ {
+ ReleaseCurrentEpoch();
+ _ = Thread.Yield();
+ AcquireCurrentEpoch();
+
+ config = clusterProvider.clusterManager.CurrentConfig;
+ }
+ while (config.GetState(slot) is SlotState.IMPORTING or SlotState.MIGRATING);
+ }
}
- ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Span keys, bool readOnly, byte sessionAsking, int count)
+ ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Span keys, bool readOnly, byte sessionAsking, bool isVectorSetWriteCommand, int count)
{
var _end = count < 0 ? keys.Length : count;
var slot = ArgSliceUtils.HashSlot(ref keys[0]);
- var verifyResult = SingleKeySlotVerify(ref config, ref keys[0], readOnly, sessionAsking, slot);
+ var verifyResult = SingleKeySlotVerify(ref config, ref keys[0], readOnly, sessionAsking, isVectorSetWriteCommand, slot);
for (var i = 1; i < _end; i++)
{
var _slot = ArgSliceUtils.HashSlot(ref keys[i]);
- var _verifyResult = SingleKeySlotVerify(ref config, ref keys[i], readOnly, sessionAsking, _slot);
+ var _verifyResult = SingleKeySlotVerify(ref config, ref keys[i], readOnly, sessionAsking, isVectorSetWriteCommand, _slot);
// Check if slot changes between keys
if (_slot != slot)
@@ -152,7 +182,7 @@ ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Sessi
{
ref var key = ref parseState.GetArgSliceByRef(csvi.firstKey);
var slot = ArgSliceUtils.HashSlot(ref key);
- var verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, slot);
+ var verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, csvi.isVectorSetWriteCommand, slot);
var secondKey = csvi.firstKey + csvi.step;
for (var i = secondKey; i < csvi.lastKey; i += csvi.step)
@@ -161,7 +191,7 @@ ClusterSlotVerificationResult MultiKeySlotVerify(ClusterConfig config, ref Sessi
continue;
key = ref parseState.GetArgSliceByRef(i);
var _slot = ArgSliceUtils.HashSlot(ref key);
- var _verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, _slot);
+ var _verifyResult = SingleKeySlotVerify(ref config, ref key, csvi.readOnly, csvi.sessionAsking, csvi.isVectorSetWriteCommand, _slot);
// Check if slot changes between keys
if (_slot != slot)
diff --git a/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs b/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs
index 3fe36867e9c..7bae12b778c 100644
--- a/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs
+++ b/libs/cluster/Session/SlotVerification/RespClusterIterativeSlotVerify.cs
@@ -28,14 +28,14 @@ public void ResetCachedSlotVerificationResult()
///
///
///
- public bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking)
+ public bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking, bool isVectorSetWriteCommand)
{
ClusterSlotVerificationResult verifyResult;
// If it is the first verification initialize the result cache
if (!initialized)
{
- verifyResult = SingleKeySlotVerify(ref configSnapshot, ref keySlice, readOnly, SessionAsking);
+ verifyResult = SingleKeySlotVerify(ref configSnapshot, ref keySlice, readOnly, SessionAsking, isVectorSetWriteCommand);
cachedVerificationResult = verifyResult;
initialized = true;
return verifyResult.state == SlotVerifiedState.OK;
@@ -45,7 +45,7 @@ public bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte Se
if (cachedVerificationResult.state != SlotVerifiedState.OK)
return false;
- verifyResult = SingleKeySlotVerify(ref configSnapshot, ref keySlice, readOnly, SessionAsking);
+ verifyResult = SingleKeySlotVerify(ref configSnapshot, ref keySlice, readOnly, SessionAsking, isVectorSetWriteCommand);
// Check if slot changes between keys
if (verifyResult.slot != cachedVerificationResult.slot)
diff --git a/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs
index af69ed8d2b2..d61822b1f4a 100644
--- a/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs
+++ b/libs/cluster/Session/SlotVerification/RespClusterSlotVerify.cs
@@ -95,13 +95,13 @@ private void WriteClusterSlotVerificationMessage(ClusterConfig config, ClusterSl
///
///
///
- public bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte sessionAsking, ref byte* dcurr, ref byte* dend, int count = -1)
+ public bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte sessionAsking, bool isVectorSetWriteCommand, ref byte* dcurr, ref byte* dend, int count = -1)
{
// If cluster is not enabled or a transaction is running skip slot check
if (!clusterProvider.serverOptions.EnableCluster || txnManager.state == TxnState.Running) return false;
var config = clusterProvider.clusterManager.CurrentConfig;
- var vres = MultiKeySlotVerify(config, ref keys, readOnly, sessionAsking, count);
+ var vres = MultiKeySlotVerify(config, ref keys, readOnly, sessionAsking, isVectorSetWriteCommand, count);
if (vres.state == SlotVerifiedState.OK)
return false;
diff --git a/libs/common/HashSlotUtils.cs b/libs/common/HashSlotUtils.cs
index f1811ce3a7e..67fbc4d29fd 100644
--- a/libs/common/HashSlotUtils.cs
+++ b/libs/common/HashSlotUtils.cs
@@ -10,6 +10,8 @@ namespace Garnet.common
{
public static unsafe class HashSlotUtils
{
+ public const ushort MaxHashSlot = 16_383;
+
///
/// This table is based on the CRC-16-CCITT polynomial (0x1021)
///
@@ -101,14 +103,14 @@ public static unsafe ushort HashSlot(byte* keyPtr, int ksize)
var startPtr = keyPtr;
var end = keyPtr + ksize;
- // Find first occurence of '{'
+ // Find first occurrence of '{'
while (startPtr < end && *startPtr != '{')
{
startPtr++;
}
// Return early if did not find '{'
- if (startPtr == end) return (ushort)(Hash(keyPtr, ksize) & 16383);
+ if (startPtr == end) return (ushort)(Hash(keyPtr, ksize) & MaxHashSlot);
var endPtr = startPtr + 1;
@@ -116,10 +118,10 @@ public static unsafe ushort HashSlot(byte* keyPtr, int ksize)
while (endPtr < end && *endPtr != '}') { endPtr++; }
// Return early if did not find '}' after '{'
- if (endPtr == end || endPtr == startPtr + 1) return (ushort)(Hash(keyPtr, ksize) & 16383);
+ if (endPtr == end || endPtr == startPtr + 1) return (ushort)(Hash(keyPtr, ksize) & MaxHashSlot);
// Return hash for byte sequence between brackets
- return (ushort)(Hash(startPtr + 1, (int)(endPtr - startPtr - 1)) & 16383);
+ return (ushort)(Hash(startPtr + 1, (int)(endPtr - startPtr - 1)) & MaxHashSlot);
}
}
}
\ No newline at end of file
diff --git a/libs/common/ReadOptimizedLock.cs b/libs/common/ReadOptimizedLock.cs
new file mode 100644
index 00000000000..2d3e76b8861
--- /dev/null
+++ b/libs/common/ReadOptimizedLock.cs
@@ -0,0 +1,408 @@
+// Copyright (c) Microsoft Corporation.
+// Licensed under the MIT license.
+
+using System;
+using System.Diagnostics;
+using System.Numerics;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+using System.Threading;
+
+namespace Garnet.common
+{
+ ///
+ /// Holds a set of RW-esque locks, optimized for reads.
+ ///
+ /// This was originally created for Vector Sets, but is general enough for reuse.
+ /// For Vector Sets, these are acquired and released as needed to prevent concurrent creation/deletion operations or deletion concurrent with read operations.
+ ///
+ /// These are outside of Tsavorite for re-entrancy reasons reasons.
+ ///
+ ///
+ /// This is a counter based r/w lock scheme, with a bit of biasing for cache line awareness.
+ ///
+ /// Each "key" acquires locks based on its hash.
+ /// Each hash is mapped to a range of indexes, each range is lockShardCount in length.
+ /// When acquiring a shared lock, we take one index out of the keys range and acquire a read lock.
+ /// This will block exclusive locks, but not impact other readers.
+ /// When acquiring an exclusive lock, we acquire write locks for all indexes in the key's range IN INCREASING _LOGICAL_ ORDER.
+ /// The order is necessary to avoid deadlocks.
+ /// By ensuring all exclusive locks walk "up" we guarantee no two exclusive lock acquisitions end up waiting for each other.
+ ///
+ /// Locks themselves are just ints, where a negative value indicates an exclusive lock and a positive value is the number of active readers.
+ /// Read locks are acquired optimistically, so actual lock values will fluctate above int.MinValue when an exclusive lock is held.
+ ///
+ /// The last set of optimizations is around cache lines coherency:
+ /// We assume cache lines of 64-bytes (the x86 default, which is also true for some [but not all] ARM processors) and size counters-per-core in multiples of that
+ /// We access array elements via reference, to avoid thrashing cache lines due to length checks
+ /// Each shard is placed, in so much as is possible, into a different cache line rather than grouping a hash's counts physically near each other
+ /// This will tend to allow a core to retain ownership of the same cache lines even as it moves between different hashes
+ ///
+ /// Experimentally (using some rough microbenchmarks) various optimizations are worth (on either shared or exclusive acquisiton paths):
+ /// - Split shards across cache lines : 7x (read path), 2.5x (write path)
+ /// - Fast math instead of mod and mult : 50% (read path), 20% (write path)
+ /// - Unsafe ref instead of array access: 0% (read path), 10% (write path)
+ ///
+ public struct ReadOptimizedLock
+ {
+ // Beyond 4K bytes per core we're well past "this is worth the tradeoff", so cut off then.
+ //
+ // Must be a power of 2.
+ private const int MaxPerCoreContexts = 1_024;
+
+ ///
+ /// Estimated size of cache lines on a processor.
+ ///
+ /// Generally correct for x86-derived processors, sometimes correct for ARM-derived ones.
+ ///
+ public const int CacheLineSizeBytes = 64;
+
+ [ThreadStatic]
+ private static int ProcessorHint;
+
+ private readonly int[] lockCounts;
+ private readonly int coreSelectionMask;
+ private readonly int perCoreCounts;
+ private readonly ulong perCoreCountsFastMod;
+ private readonly byte perCoreCountsMultShift;
+
+ ///
+ /// Create a new .
+ ///
+ /// accuracy impacts performance, not correctness.
+ ///
+ /// Too low and unrelated locks will end up delaying each other.
+ /// Too high and more memory than is necessary will be used.
+ ///
+ public ReadOptimizedLock(int estimatedSimultaneousActiveLockers)
+ {
+ Debug.Assert(estimatedSimultaneousActiveLockers > 0);
+
+ // ~1 per core
+ var coreCount = (int)BitOperations.RoundUpToPowerOf2((uint)Environment.ProcessorCount);
+ coreSelectionMask = coreCount - 1;
+
+ // Use estimatedSimultaneousActiveLockers to determine number of shards per lock.
+ //
+ // We scale up to a whole multiple of CacheLineSizeBytes to reduce cache line thrashing.
+ //
+ // We scale to a power of 2 to avoid divisions (and some multiplies) in index calculation.
+ perCoreCounts = estimatedSimultaneousActiveLockers;
+ if (perCoreCounts % (CacheLineSizeBytes / sizeof(int)) != 0)
+ {
+ perCoreCounts += (CacheLineSizeBytes / sizeof(int)) - (perCoreCounts % (CacheLineSizeBytes / sizeof(int)));
+ }
+ Debug.Assert(perCoreCounts % (CacheLineSizeBytes / sizeof(int)) == 0, "Each core should be whole cache lines of data");
+
+ perCoreCounts = (int)BitOperations.RoundUpToPowerOf2((uint)perCoreCounts);
+
+ // Put an upper bound of ~1 page worth of locks per core (which is still quite high).
+ //
+ // For the largest realistic machines out there (384 cores) this will put us at around ~2M of lock data, max.
+ if (perCoreCounts is <= 0 or > MaxPerCoreContexts)
+ {
+ perCoreCounts = MaxPerCoreContexts;
+ }
+
+ // Pre-calculate an alternative to %, as that division will be in the hot path
+ perCoreCountsFastMod = (ulong.MaxValue / (uint)perCoreCounts) + 1;
+
+ // Avoid two multiplies in the hot path
+ perCoreCountsMultShift = (byte)BitOperations.Log2((uint)perCoreCounts);
+
+ var numInts = coreCount * perCoreCounts;
+ lockCounts = new int[numInts];
+ }
+
+ ///
+ /// Take a hash and a _hint_ about the current processor and determine which count should be used.
+ ///
+ /// Walking from 0 to ( + 1) [exclusive] will return
+ /// all possible counts for a given hash.
+ ///
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public readonly int CalculateIndex(long hashLong, int currentProcessorHint)
+ {
+ // Throw away half the top half of the hash
+ //
+ // This set of locks will be small enough that the extra bits shoulnd't matter
+ var hash = (int)hashLong;
+
+ // Hint might be out of range, so force it into the space we expect
+ var currentProcessor = currentProcessorHint & coreSelectionMask;
+
+ var startOfCoreCounts = currentProcessor << perCoreCountsMultShift;
+
+ // Avoid doing a division in the hot path
+ // Based on: https://github.com/dotnet/runtime/blob/3a95842304008b9ca84c14b4bec9ec99ed5802db/src/libraries/System.Private.CoreLib/src/System/Collections/HashHelpers.cs#L99
+ var hashOffset = (uint)(((((perCoreCountsFastMod * (uint)hash) >> 32) + 1) << perCoreCountsMultShift) >> 32);
+
+ Debug.Assert(hashOffset == ((uint)hash % perCoreCounts), "Replacing mod with multiplies failed");
+
+ var ix = (int)(startOfCoreCounts + hashOffset);
+
+ Debug.Assert(ix >= 0 && ix < lockCounts.Length, "About to do something out of bounds");
+
+ return ix;
+ }
+
+ ///
+ /// Attempt to acquire a shared lock for the given hash.
+ ///
+ /// Will block exclusive locks until released.
+ ///
+ public readonly bool TryAcquireSharedLock(long hash, out int lockToken)
+ {
+ var ix = CalculateIndex(hash, GetProcessorHint());
+
+ ref var acquireRef = ref Unsafe.Add(ref MemoryMarshal.GetArrayDataReference(lockCounts), ix);
+
+ var res = Interlocked.Increment(ref acquireRef);
+ if (res < 0)
+ {
+ // Exclusively locked
+ _ = Interlocked.Decrement(ref acquireRef);
+ Unsafe.SkipInit(out lockToken);
+ return false;
+ }
+
+ lockToken = ix;
+ return true;
+ }
+
+ ///
+ /// Acquire a shared lock for the given hash, blocking until that succeeds.
+ ///
+ /// Will block exclusive locks until released.
+ ///
+ public readonly void AcquireSharedLock(long hash, out int lockToken)
+ {
+ var ix = CalculateIndex(hash, GetProcessorHint());
+
+ ref var acquireRef = ref Unsafe.Add(ref MemoryMarshal.GetArrayDataReference(lockCounts), ix);
+
+ while (true)
+ {
+ var res = Interlocked.Increment(ref acquireRef);
+ if (res < 0)
+ {
+ // Exclusively locked
+ _ = Interlocked.Decrement(ref acquireRef);
+
+ // Spin until we can grab this one
+ _ = Thread.Yield();
+ }
+ else
+ {
+ lockToken = ix;
+ return;
+ }
+ }
+ }
+
+ ///
+ /// Release a lock previously acquired with or .
+ ///
+ public readonly void ReleaseSharedLock(int lockToken)
+ {
+ Debug.Assert(lockToken >= 0 && lockToken < lockCounts.Length, "Invalid lock token");
+
+ ref var releaseRef = ref Unsafe.Add(ref MemoryMarshal.GetArrayDataReference(lockCounts), lockToken);
+
+ _ = Interlocked.Decrement(ref releaseRef);
+ }
+
+ ///
+ /// Attempt to acquire an exclusive lock for the given hash.
+ ///
+ /// Will block all other locks until released.
+ ///
+ public readonly bool TryAcquireExclusiveLock(long hash, out int lockToken)
+ {
+ ref var countRef = ref MemoryMarshal.GetArrayDataReference(lockCounts);
+
+ var coreCount = coreSelectionMask + 1;
+ for (var i = 0; i < coreCount; i++)
+ {
+ var acquireIx = CalculateIndex(hash, i);
+ ref var acquireRef = ref Unsafe.Add(ref countRef, acquireIx);
+
+ if (Interlocked.CompareExchange(ref acquireRef, int.MinValue, 0) != 0)
+ {
+ // Failed, release previously acquired
+ for (var j = 0; j < i; j++)
+ {
+ var releaseIx = CalculateIndex(hash, j);
+
+ ref var releaseRef = ref Unsafe.Add(ref countRef, releaseIx);
+ while (Interlocked.CompareExchange(ref releaseRef, 0, int.MinValue) != int.MinValue)
+ {
+ // Optimistic shared lock got us, back off and try again
+ _ = Thread.Yield();
+ }
+ }
+
+ Unsafe.SkipInit(out lockToken);
+ return false;
+ }
+ }
+
+ // Successfully acquired all shards exclusively
+
+ // Throwing away half the hash shouldn't affect correctness since we do the same thing when processing the full hash
+ lockToken = (int)hash;
+
+ return true;
+ }
+
+ ///
+ /// Acquire an exclusive lock for the given hash, blocking until that succeeds.
+ ///
+ /// Will block all other locks until released.
+ ///
+ public readonly void AcquireExclusiveLock(long hash, out int lockToken)
+ {
+ ref var countRef = ref MemoryMarshal.GetArrayDataReference(lockCounts);
+
+ var coreCount = coreSelectionMask + 1;
+ for (var i = 0; i < coreCount; i++)
+ {
+ var acquireIx = CalculateIndex(hash, i);
+
+ ref var acquireRef = ref Unsafe.Add(ref countRef, acquireIx);
+ while (Interlocked.CompareExchange(ref acquireRef, int.MinValue, 0) != 0)
+ {
+ // Optimistic shared lock got us, or conflict with some other excluive lock acquisition
+ //
+ // Backoff and try again
+ _ = Thread.Yield();
+ }
+ }
+
+ // Throwing away half the hash shouldn't affect correctness since we do the same thing when processing the full hash
+ lockToken = (int)hash;
+ }
+
+ ///
+ /// Release a lock previously acquired with , , or .
+ ///
+ public readonly void ReleaseExclusiveLock(int lockToken)
+ {
+ // The lockToken is a hash, so no range check here
+
+ ref var countRef = ref MemoryMarshal.GetArrayDataReference(lockCounts);
+
+ var hash = lockToken;
+
+ var coreCount = coreSelectionMask + 1;
+ for (var i = 0; i < coreCount; i++)
+ {
+ var releaseIx = CalculateIndex(hash, i);
+
+ ref var releaseRef = ref Unsafe.Add(ref countRef, releaseIx);
+ while (Interlocked.CompareExchange(ref releaseRef, 0, int.MinValue) != int.MinValue)
+ {
+ // Optimistic shared lock got us, back off and try again
+ _ = Thread.Yield();
+ }
+ }
+ }
+
+ ///
+ /// Attempt to promote a shared lock previously acquired via or to an exclusive lock.
+ ///
+ /// If successful, will block all other locks until released.
+ ///
+ /// If successful, must be released with .
+ ///
+ /// If unsuccessful, shared lock will still be held and must be released with .
+ ///
+ public readonly bool TryPromoteSharedLock(long hash, int lockToken, out int newLockToken)
+ {
+ Debug.Assert(Interlocked.CompareExchange(ref lockCounts[lockToken], 0, 0) > 0, "Illegal call when not holding shard lock");
+
+ Debug.Assert(lockToken >= 0 && lockToken < lockCounts.Length, "Invalid lock token");
+
+ ref var countRef = ref MemoryMarshal.GetArrayDataReference(lockCounts);
+
+ var coreCount = coreSelectionMask + 1;
+ for (var i = 0; i < coreCount; i++)
+ {
+ var acquireIx = CalculateIndex(hash, i);
+ ref var acquireRef = ref Unsafe.Add(ref countRef, acquireIx);
+
+ if (acquireIx == lockToken)
+ {
+ // Do the promote
+ if (Interlocked.CompareExchange(ref acquireRef, int.MinValue, 1) != 1)
+ {
+ // Failed, release previously acquired all of which are exclusive locks
+ for (var j = 0; j < i; j++)
+ {
+ var releaseIx = CalculateIndex(hash, j);
+
+ ref var releaseRef = ref Unsafe.Add(ref countRef, releaseIx);
+ while (Interlocked.CompareExchange(ref releaseRef, 0, int.MinValue) != int.MinValue)
+ {
+ // Optimistic shared lock got us, back off and try again
+ _ = Thread.Yield();
+ }
+ }
+
+ // Note we're still holding the shared lock here
+ Unsafe.SkipInit(out newLockToken);
+ return false;
+ }
+ }
+ else
+ {
+ // Otherwise attempt an exclusive acquire
+ if (Interlocked.CompareExchange(ref acquireRef, int.MinValue, 0) != 0)
+ {
+ // Failed, release previously acquired - one of which MIGHT be the shared lock
+ for (var j = 0; j < i; j++)
+ {
+ var releaseIx = CalculateIndex(hash, j);
+ var releaseTargetValue = releaseIx == lockToken ? 1 : 0;
+
+ ref var releaseRef = ref Unsafe.Add(ref countRef, releaseIx);
+ while (Interlocked.CompareExchange(ref releaseRef, releaseTargetValue, int.MinValue) != int.MinValue)
+ {
+ // Optimistic shared lock got us, back off and try again
+ _ = Thread.Yield();
+ }
+ }
+
+ // Note we're still holding the shared lock here
+ Unsafe.SkipInit(out newLockToken);
+ return false;
+ }
+ }
+ }
+
+ // Throwing away half the hash shouldn't affect correctness since we do the same thing when processing the full hash
+ newLockToken = (int)hash;
+ return true;
+ }
+
+ ///
+ /// Get a somewhat-correlated-to-processor value.
+ ///
+ /// While we could use , that isn't fast on all platforms.
+ ///
+ /// For our purposes, we just need something that will tend to keep different active processors
+ /// from touching each other. ManagedThreadId works well enough.
+ ///
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ private static int GetProcessorHint()
+ {
+ var ret = ProcessorHint;
+ if (ret == 0)
+ {
+ ProcessorHint = ret = Environment.CurrentManagedThreadId;
+ }
+
+ return ret;
+ }
+ }
+}
\ No newline at end of file
diff --git a/libs/common/RespReadUtils.cs b/libs/common/RespReadUtils.cs
index 92c41ec4739..1202e8c0e09 100644
--- a/libs/common/RespReadUtils.cs
+++ b/libs/common/RespReadUtils.cs
@@ -1341,5 +1341,40 @@ public static bool TryReadInfinity(ReadOnlySpan value, out double number)
number = default;
return false;
}
+
+ ///
+ /// Parses "[+/-]inf" string and returns float.PositiveInfinity/float.NegativeInfinity respectively.
+ /// If string is not an infinity, parsing fails.
+ ///
+ /// input data
+ /// If parsing was successful,contains positive or negative infinity
+ /// True is infinity was read, false otherwise
+ public static bool TryReadInfinity(ReadOnlySpan value, out float number)
+ {
+ if (value.Length == 3)
+ {
+ if (value.EqualsUpperCaseSpanIgnoringCase(RespStrings.INFINITY))
+ {
+ number = float.PositiveInfinity;
+ return true;
+ }
+ }
+ else if (value.Length == 4)
+ {
+ if (value.EqualsUpperCaseSpanIgnoringCase(RespStrings.POS_INFINITY, true))
+ {
+ number = float.PositiveInfinity;
+ return true;
+ }
+ else if (value.EqualsUpperCaseSpanIgnoringCase(RespStrings.NEG_INFINITY, true))
+ {
+ number = float.NegativeInfinity;
+ return true;
+ }
+ }
+
+ number = default;
+ return false;
+ }
}
}
\ No newline at end of file
diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs
index 2ba5b3302d4..ba87d1c59a9 100644
--- a/libs/host/Configuration/Options.cs
+++ b/libs/host/Configuration/Options.cs
@@ -666,6 +666,9 @@ public IEnumerable LuaAllowedFunctions
[Option("cluster-replica-resume-with-data", Required = false, HelpText = "If a Cluster Replica resumes with data, allow it to be served prior to a Primary being available")]
public bool ClusterReplicaResumeWithData { get; set; }
+ [Option("enable-vector-set-preview", Required = false, HelpText = "Enable Vector Sets (preview) - this feature (and associated commands) are incomplete, unstable, and subject to change while still in preview")]
+ public bool EnableVectorSetPreview { get; set; }
+
///
/// This property contains all arguments that were not parsed by the command line argument parser
///
@@ -952,6 +955,7 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null)
ExpiredKeyDeletionScanFrequencySecs = ExpiredKeyDeletionScanFrequencySecs,
ClusterReplicationReestablishmentTimeout = ClusterReplicationReestablishmentTimeout,
ClusterReplicaResumeWithData = ClusterReplicaResumeWithData,
+ EnableVectorSetPreview = EnableVectorSetPreview,
};
}
diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs
index 2d12f43a0d4..05103d39106 100644
--- a/libs/host/GarnetServer.cs
+++ b/libs/host/GarnetServer.cs
@@ -303,9 +303,18 @@ private GarnetDatabase CreateDatabase(int dbId, GarnetServerOptions serverOption
var store = CreateMainStore(dbId, clusterFactory, out var epoch, out var stateMachineDriver);
var objectStore = CreateObjectStore(dbId, clusterFactory, customCommandManager, epoch, stateMachineDriver, out var objectStoreSizeTracker);
var (aofDevice, aof) = CreateAOF(dbId);
+
+ var vectorManager = new VectorManager(
+ serverOptions.EnableVectorSetPreview,
+ dbId,
+ () => Provider.GetSession(WireFormat.ASCII, null),
+ loggerFactory
+ );
+
return new GarnetDatabase(dbId, store, objectStore, epoch, stateMachineDriver, objectStoreSizeTracker,
aofDevice, aof, serverOptions.AdjustedIndexMaxCacheLines == 0,
- serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0);
+ serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0,
+ vectorManager);
}
private void LoadModules(CustomCommandManager customCommandManager)
diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf
index a14f15f5052..0b0d025144a 100644
--- a/libs/host/defaults.conf
+++ b/libs/host/defaults.conf
@@ -447,5 +447,8 @@
"ClusterReplicationReestablishmentTimeout": 0,
/* If a Cluster Replica has on disk checkpoints or AOF, if that data should be loaded on restart instead of waiting for a Primary to sync with */
- "ClusterReplicaResumeWithData": false
+ "ClusterReplicaResumeWithData": false,
+
+ /* Enable Vector Sets (preview) - this feature (and associated commands) are incomplete, unstable, and subject to change while still in preview */
+ "EnableVectorSetPreview": false
}
\ No newline at end of file
diff --git a/libs/resources/RespCommandsDocs.json b/libs/resources/RespCommandsDocs.json
index be77703a3ed..049b46fd80f 100644
--- a/libs/resources/RespCommandsDocs.json
+++ b/libs/resources/RespCommandsDocs.json
@@ -7719,6 +7719,204 @@
"Group": "Transactions",
"Complexity": "O(1)"
},
+ {
+ "Command": "VADD",
+ "Name": "VADD",
+ "Summary": "Add a new element into the vector set.",
+ "Group": "Vector",
+ "Complexity": "O(log(N))",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VCARD",
+ "Name": "VCARD",
+ "Summary": "Return the number of elements in a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VDIM",
+ "Name": "VDIM",
+ "Summary": "Return the number of dimensions in a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VEMB",
+ "Name": "VEMB",
+ "Summary": "Return the approximate vector associated with an element in a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VGETATTR",
+ "Name": "VGETATTR",
+ "Summary": "Return the JSON attributes associated with the element in the vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VINFO",
+ "Name": "VINFO",
+ "Summary": "Return details about a vector set, including dimensions, quantization, and structure.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VISMEMBER",
+ "Name": "VISMEMBER",
+ "Summary": "Determines whether a member belongs to vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ },
+ {
+ "TypeDiscriminator": "RespCommandBasicArgument",
+ "Name": "ELEMENT",
+ "DisplayText": "element",
+ "Type": "String"
+ }
+ ]
+ },
+ {
+ "Command": "VLINKS",
+ "Name": "VLINKS",
+ "Summary": "Return the neighbors of an element in a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VRANDMEMBER",
+ "Name": "VRANDMEMBER",
+ "Summary": "Return some number of random elements from a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VREM",
+ "Name": "VREM",
+ "Summary": "Remove an element from a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(log(N))",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VSETATTR",
+ "Name": "VSETATTR",
+ "Summary": "Store attributes alongside a member of a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(1)",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
+ {
+ "Command": "VSIM",
+ "Name": "VSIM",
+ "Summary": "Return elements similar to a given vector or existing element of a vector set.",
+ "Group": "Vector",
+ "Complexity": "O(log(N))",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "KEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "KeySpecIndex": 0
+ }
+ ]
+ },
{
"Command": "WATCH",
"Name": "WATCH",
diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json
index daa1acd29d3..40aa9686505 100644
--- a/libs/resources/RespCommandsInfo.json
+++ b/libs/resources/RespCommandsInfo.json
@@ -811,6 +811,14 @@
"Flags": "Admin, NoMulti, NoScript",
"AclCategories": "Admin, Dangerous, Slow, Garnet"
},
+ {
+ "Command": "CLUSTER_RESERVE",
+ "Name": "CLUSTER|RESERVE",
+ "IsInternal": true,
+ "Arity": 4,
+ "Flags": "Admin, NoMulti, NoScript",
+ "AclCategories": "Admin, Dangerous, Garnet"
+ },
{
"Command": "CLUSTER_MTASKS",
"Name": "CLUSTER|MTASKS",
@@ -5093,6 +5101,306 @@
"Flags": "Fast, Loading, NoScript, Stale, AllowBusy",
"AclCategories": "Fast, Transaction"
},
+ {
+ "Command": "VADD",
+ "Name": "VADD",
+ "Arity": -1,
+ "Flags": "DenyOom, Write, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Vector, Write",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RW, Insert"
+ }
+ ]
+ },
+ {
+ "Command": "VCARD",
+ "Name": "VCARD",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VDIM",
+ "Name": "VDIM",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VEMB",
+ "Name": "VEMB",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VGETATTR",
+ "Name": "VGETATTR",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VINFO",
+ "Name": "VINFO",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VISMEMBER",
+ "Name": "VISMEMBER",
+ "Arity": 3,
+ "Flags": "Fast, ReadOnly",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VLINKS",
+ "Name": "VLINKS",
+ "Arity": -1,
+ "Flags": "Fast, ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VRANDMEMBER",
+ "Name": "VRANDMEMBER",
+ "Arity": -1,
+ "Flags": "ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Slow, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
+ {
+ "Command": "VREM",
+ "Name": "VREM",
+ "Arity": -1,
+ "Flags": "Write, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Slow, Write, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RW, Delete"
+ }
+ ]
+ },
+ {
+ "Command": "VSETATTR",
+ "Name": "VSETATTR",
+ "Arity": -1,
+ "Flags": "Fast, Write, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Fast, Write, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RW, Insert"
+ }
+ ]
+ },
+ {
+ "Command": "VSIM",
+ "Name": "VSIM",
+ "Arity": -1,
+ "Flags": "ReadOnly, Module",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Slow, Read, Vector",
+ "KeySpecifications": [
+ {
+ "BeginSearch": {
+ "TypeDiscriminator": "BeginSearchIndex",
+ "Index": 1
+ },
+ "FindKeys": {
+ "TypeDiscriminator": "FindKeysRange",
+ "LastKey": 0,
+ "KeyStep": 1,
+ "Limit": 0
+ },
+ "Flags": "RO"
+ }
+ ]
+ },
{
"Command": "WATCH",
"Name": "WATCH",
diff --git a/libs/server/ACL/ACLParser.cs b/libs/server/ACL/ACLParser.cs
index 621d7a44e8e..2ee3297867c 100644
--- a/libs/server/ACL/ACLParser.cs
+++ b/libs/server/ACL/ACLParser.cs
@@ -33,6 +33,7 @@ class ACLParser
["stream"] = RespAclCategories.Stream,
["string"] = RespAclCategories.String,
["transaction"] = RespAclCategories.Transaction,
+ ["vector"] = RespAclCategories.Vector,
["write"] = RespAclCategories.Write,
["garnet"] = RespAclCategories.Garnet,
["custom"] = RespAclCategories.Custom,
diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs
index f4c21c5e7f5..766d83864d8 100644
--- a/libs/server/AOF/AofProcessor.cs
+++ b/libs/server/AOF/AofProcessor.cs
@@ -34,6 +34,7 @@ public sealed unsafe partial class AofProcessor
private readonly SessionParseState parseState;
int activeDbId;
+ VectorManager activeVectorManager;
///
/// Set ReadWriteSession on the cluster session (NOTE: used for replaying stored procedures only)
@@ -57,6 +58,9 @@ public sealed unsafe partial class AofProcessor
readonly ILogger logger;
+ readonly StoreWrapper replayAofStoreWrapper;
+ readonly IClusterProvider clusterProvider;
+
MemoryResult output;
///
@@ -70,10 +74,11 @@ public AofProcessor(
{
this.storeWrapper = storeWrapper;
- var replayAofStoreWrapper = new StoreWrapper(storeWrapper, recordToAof);
+ replayAofStoreWrapper = new StoreWrapper(storeWrapper, recordToAof);
+ this.clusterProvider = clusterProvider;
this.activeDbId = 0;
- this.respServerSession = new RespServerSession(0, networkSender: null, storeWrapper: replayAofStoreWrapper, subscribeBroker: null, authenticator: null, enableScripts: false, clusterProvider: clusterProvider);
+ this.respServerSession = ObtainServerSession();
// Switch current contexts to match the default database
SwitchActiveDatabaseContext(storeWrapper.DefaultDatabase, true);
@@ -90,6 +95,9 @@ public AofProcessor(
this.logger = logger;
}
+ private RespServerSession ObtainServerSession()
+ => new(0, networkSender: null, storeWrapper: replayAofStoreWrapper, subscribeBroker: null, authenticator: null, enableScripts: false, clusterProvider: clusterProvider);
+
///
/// Dispose
///
@@ -191,6 +199,12 @@ public unsafe void ProcessAofRecordInternal(byte* ptr, int length, bool asReplic
AofHeader header = *(AofHeader*)ptr;
isCheckpointStart = false;
+ // Aggressively do not move data if VADD are being replayed
+ if (header.opType != AofEntryType.StoreRMW)
+ {
+ activeVectorManager.WaitForVectorOperationsToComplete();
+ }
+
if (inflightTxns.ContainsKey(header.sessionID))
{
switch (header.opType)
@@ -332,6 +346,14 @@ private unsafe bool ReplayOp(byte* entryPtr, int length, bool replayAsReplica)
{
AofHeader header = *(AofHeader*)entryPtr;
+ // StoreRMW can queue VADDs onto different threads
+ // but everything else needs to WAIT for those to complete
+ // otherwise we might loose consistency
+ if (header.opType != AofEntryType.StoreRMW)
+ {
+ activeVectorManager.WaitForVectorOperationsToComplete();
+ }
+
// Skips (1) entries with versions that were part of prior checkpoint; and (2) future entries in fuzzy region
if (SkipRecord(entryPtr, length, replayAsReplica)) return false;
@@ -341,10 +363,10 @@ private unsafe bool ReplayOp(byte* entryPtr, int length, bool replayAsReplica)
StoreUpsert(basicContext, storeInput, entryPtr);
break;
case AofEntryType.StoreRMW:
- StoreRMW(basicContext, storeInput, entryPtr);
+ StoreRMW(basicContext, storeInput, activeVectorManager, respServerSession, ObtainServerSession, entryPtr);
break;
case AofEntryType.StoreDelete:
- StoreDelete(basicContext, entryPtr);
+ StoreDelete(basicContext, activeVectorManager, respServerSession.storageSession, entryPtr);
break;
case AofEntryType.ObjectStoreRMW:
ObjectStoreRMW(objectStoreBasicContext, objectStoreInput, entryPtr, bufferPtr, buffer.Length);
@@ -396,6 +418,8 @@ private void SwitchActiveDatabaseContext(GarnetDatabase db, bool initialSetup =
objectStoreBasicContext = objectStoreSession.BasicContext;
this.activeDbId = db.Id;
}
+
+ activeVectorManager = db.VectorManager;
}
static void StoreUpsert(BasicContext basicContext,
@@ -419,7 +443,14 @@ static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr)
+ static void StoreRMW(
+ BasicContext basicContext,
+ RawStringInput storeInput,
+ VectorManager vectorManager,
+ RespServerSession currentSession,
+ Func obtainServerSession,
+ byte* ptr
+ )
{
var curr = ptr + sizeof(AofHeader);
ref var key = ref Unsafe.AsRef(curr);
@@ -428,21 +459,52 @@ static void StoreRMW(BasicContext basicContext, byte* ptr)
+ static void StoreDelete(
+ BasicContext basicContext,
+ VectorManager vectorManager,
+ StorageSession storageSession,
+ byte* ptr)
{
ref var key = ref Unsafe.AsRef(ptr + sizeof(AofHeader));
- basicContext.Delete(ref key);
+ var res = basicContext.Delete(ref key);
+
+ if (res.IsCanceled)
+ {
+ // Might be a vector set
+ res = vectorManager.TryDeleteVectorSet(storageSession, ref key);
+ if (res.IsPending)
+ _ = basicContext.CompletePending(true);
+ }
}
static void ObjectStoreUpsert(BasicContext basicContext,
diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs
index 09d23aad563..8f9e19200e9 100644
--- a/libs/server/API/GarnetApi.cs
+++ b/libs/server/API/GarnetApi.cs
@@ -21,9 +21,10 @@ namespace Garnet.server
///
/// Garnet API implementation
///
- public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
+ public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
where TContext : ITsavoriteContext
where TObjectContext : ITsavoriteContext
+ where TVectorContext : ITsavoriteContext
{
readonly StorageSession storageSession;
TContext context;
@@ -48,8 +49,12 @@ public void WATCH(byte[] key, StoreType type)
#region GET
///
- public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output)
- => storageSession.GET(ref key, ref input, ref output, ref context);
+ public GarnetStatus GET(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output)
+ {
+ var asSpanByte = key.SpanByte;
+
+ return storageSession.GET(ref asSpanByte, ref input, ref output, ref context);
+ }
///
public GarnetStatus GET_WithPending(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output, long ctx, out bool pending)
@@ -68,7 +73,9 @@ public unsafe GarnetStatus GETForMemoryResult(ArgSlice key, out MemoryResult
public unsafe GarnetStatus GET(ArgSlice key, out ArgSlice value)
- => storageSession.GET(key, out value, ref context);
+ {
+ return storageSession.GET(key, out value, ref context);
+ }
///
public GarnetStatus GET(byte[] key, out GarnetObjectStoreOutput value)
@@ -118,33 +125,52 @@ public GarnetStatus PEXPIRETIME(ref SpanByte key, StoreType storeType, ref SpanB
#endregion
#region SET
- ///
+
public GarnetStatus SET(ref SpanByte key, ref SpanByte value)
- => storageSession.SET(ref key, ref value, ref context);
+ => storageSession.SET(ref key, ref value, ref context);
///
- public GarnetStatus SET(ref SpanByte key, ref RawStringInput input, ref SpanByte value)
- => storageSession.SET(ref key, ref input, ref value, ref context);
+ public GarnetStatus SET(ArgSlice key, ref RawStringInput input, ref SpanByte value)
+ {
+ var asSpanByte = key.SpanByte;
- ///
- public GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input)
- => storageSession.SET_Conditional(ref key, ref input, ref context);
+ return storageSession.SET(ref asSpanByte, ref input, ref value, ref context);
+ }
///
public GarnetStatus DEL_Conditional(ref SpanByte key, ref RawStringInput input)
=> storageSession.DEL_Conditional(ref key, ref input, ref context);
///
- public GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output)
- => storageSession.SET_Conditional(ref key, ref input, ref output, ref context);
+ public GarnetStatus SET_Conditional(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output)
+ {
+ var asSpanByte = key.SpanByte;
+
+ return storageSession.SET_Conditional(ref asSpanByte, ref input, ref output, ref context);
+ }
+
+ ///
+ public GarnetStatus SET_Conditional(ArgSlice key, ref RawStringInput input)
+ {
+ var asSpanByte = key.SpanByte;
+
+ return storageSession.SET_Conditional(ref asSpanByte, ref input, ref context);
+ }
///
public GarnetStatus SET(ArgSlice key, Memory value)
- => storageSession.SET(key, value, ref context);
+ {
+ return storageSession.SET(key, value, ref context);
+ }
///
public GarnetStatus SET(ArgSlice key, ArgSlice value)
- => storageSession.SET(key, value, ref context);
+ {
+ var asSpanByte = key.SpanByte;
+ var valSpanByte = value.SpanByte;
+
+ return storageSession.SET(ref asSpanByte, ref valSpanByte, ref context);
+ }
///
public GarnetStatus SET(byte[] key, IGarnetObject value)
@@ -302,7 +328,7 @@ public GarnetStatus DELETE(ArgSlice key, StoreType storeType = StoreType.All)
///
public GarnetStatus DELETE(ref SpanByte key, StoreType storeType = StoreType.All)
- => storageSession.DELETE(ref key, storeType, ref context, ref objectContext);
+ => storageSession.DELETE(ref key, storeType, ref context, ref objectContext);
///
public GarnetStatus DELETE(byte[] key, StoreType storeType = StoreType.All)
@@ -482,5 +508,33 @@ public int GetScratchBufferOffset()
public bool ResetScratchBuffer(int offset)
=> storageSession.scratchBufferBuilder.ResetScratchBuffer(offset);
#endregion
+
+ #region VectorSet commands
+
+ ///
+ public unsafe GarnetStatus VectorSetAdd(ArgSlice key, int reduceDims, VectorValueType valueType, ArgSlice values, ArgSlice element, VectorQuantType quantizer, int buildExplorationFactor, ArgSlice attributes, int numLinks, out VectorManagerResult result, out ReadOnlySpan errorMsg)
+ => storageSession.VectorSetAdd(SpanByte.FromPinnedPointer(key.ptr, key.length), reduceDims, valueType, values, element, quantizer, buildExplorationFactor, attributes, numLinks, out result, out errorMsg);
+
+ ///
+ public unsafe GarnetStatus VectorSetRemove(ArgSlice key, ArgSlice element)
+ => storageSession.VectorSetRemove(SpanByte.FromPinnedPointer(key.ptr, key.length), SpanByte.FromPinnedPointer(element.ptr, element.length));
+
+ ///
+ public unsafe GarnetStatus VectorSetValueSimilarity(ArgSlice key, VectorValueType valueType, ArgSlice values, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result)
+ => storageSession.VectorSetValueSimilarity(SpanByte.FromPinnedPointer(key.ptr, key.length), valueType, values, count, delta, searchExplorationFactor, filter.ReadOnlySpan, maxFilteringEffort, includeAttributes, ref outputIds, out outputIdFormat, ref outputDistances, ref outputAttributes, out result);
+
+ ///
+ public unsafe GarnetStatus VectorSetElementSimilarity(ArgSlice key, ArgSlice element, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result)
+ => storageSession.VectorSetElementSimilarity(SpanByte.FromPinnedPointer(key.ptr, key.length), element.ReadOnlySpan, count, delta, searchExplorationFactor, filter.ReadOnlySpan, maxFilteringEffort, includeAttributes, ref outputIds, out outputIdFormat, ref outputDistances, ref outputAttributes, out result);
+
+ ///
+ public unsafe GarnetStatus VectorSetEmbedding(ArgSlice key, ArgSlice element, ref SpanByteAndMemory outputDistances)
+ => storageSession.VectorSetEmbedding(SpanByte.FromPinnedPointer(key.ptr, key.length), element.ReadOnlySpan, ref outputDistances);
+
+ ///
+ public unsafe GarnetStatus VectorSetDimensions(ArgSlice key, out int dimensions)
+ => storageSession.VectorSetDimensions(SpanByte.FromPinnedPointer(key.ptr, key.length), out dimensions);
+
+ #endregion
}
}
\ No newline at end of file
diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs
index b0a72473b8e..9ba483e08d7 100644
--- a/libs/server/API/GarnetApiObjectCommands.cs
+++ b/libs/server/API/GarnetApiObjectCommands.cs
@@ -16,9 +16,10 @@ namespace Garnet.server
///
/// Garnet API implementation
///
- public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
+ public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
where TContext : ITsavoriteContext
where TObjectContext : ITsavoriteContext
+ where TVectorContext : ITsavoriteContext
{
#region SortedSet Methods
diff --git a/libs/server/API/GarnetWatchApi.cs b/libs/server/API/GarnetWatchApi.cs
index ac68e97e66f..ff0f3a2063f 100644
--- a/libs/server/API/GarnetWatchApi.cs
+++ b/libs/server/API/GarnetWatchApi.cs
@@ -23,10 +23,10 @@ public GarnetWatchApi(TGarnetApi garnetApi)
#region GET
///
- public GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output)
+ public GarnetStatus GET(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output)
{
- garnetApi.WATCH(new ArgSlice(ref key), StoreType.Main);
- return garnetApi.GET(ref key, ref input, ref output);
+ garnetApi.WATCH(key, StoreType.Main);
+ return garnetApi.GET(key, ref input, ref output);
}
///
@@ -647,5 +647,35 @@ public bool ResetScratchBuffer(int offset)
=> garnetApi.ResetScratchBuffer(offset);
#endregion
+
+ #region Vector Sets
+ ///
+ public GarnetStatus VectorSetValueSimilarity(ArgSlice key, VectorValueType valueType, ArgSlice value, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result)
+ {
+ garnetApi.WATCH(key, StoreType.Main);
+ return garnetApi.VectorSetValueSimilarity(key, valueType, value, count, delta, searchExplorationFactor, filter, maxFilteringEffort, includeAttributes, ref outputIds, out outputIdFormat, ref outputDistances, ref outputAttributes, out result);
+ }
+
+ ///
+ public GarnetStatus VectorSetElementSimilarity(ArgSlice key, ArgSlice element, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result)
+ {
+ garnetApi.WATCH(key, StoreType.Main);
+ return garnetApi.VectorSetElementSimilarity(key, element, count, delta, searchExplorationFactor, filter, maxFilteringEffort, includeAttributes, ref outputIds, out outputIdFormat, ref outputDistances, ref outputAttributes, out result);
+ }
+
+ ///
+ public GarnetStatus VectorSetEmbedding(ArgSlice key, ArgSlice element, ref SpanByteAndMemory outputDistances)
+ {
+ garnetApi.WATCH(key, StoreType.Main);
+ return garnetApi.VectorSetEmbedding(key, element, ref outputDistances);
+ }
+
+ ///
+ public GarnetStatus VectorSetDimensions(ArgSlice key, out int dimensions)
+ {
+ garnetApi.WATCH(key, StoreType.Main);
+ return garnetApi.VectorSetDimensions(key, out dimensions);
+ }
+ #endregion
}
}
\ No newline at end of file
diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs
index a78ac22f556..f81597912fa 100644
--- a/libs/server/API/IGarnetApi.cs
+++ b/libs/server/API/IGarnetApi.cs
@@ -26,17 +26,12 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi
///
/// SET
///
- GarnetStatus SET(ref SpanByte key, ref SpanByte value);
-
- ///
- /// SET
- ///
- GarnetStatus SET(ref SpanByte key, ref RawStringInput input, ref SpanByte value);
+ GarnetStatus SET(ArgSlice key, ref RawStringInput input, ref SpanByte value);
///
/// SET Conditional
///
- GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input);
+ GarnetStatus SET_Conditional(ArgSlice key, ref RawStringInput input);
///
/// DEL Conditional
@@ -46,7 +41,7 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi
///
/// SET Conditional
///
- GarnetStatus SET_Conditional(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output);
+ GarnetStatus SET_Conditional(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output);
///
/// SET
@@ -1206,6 +1201,18 @@ GarnetStatus GeoSearchStore(ArgSlice key, ArgSlice destinationKey, ref GeoSearch
GarnetStatus HyperLogLogMerge(ref RawStringInput input, out bool error);
#endregion
+
+ #region VectorSet Methods
+ ///
+ /// Adds to (and may create) a vector set with the given parameters.
+ ///
+ GarnetStatus VectorSetAdd(ArgSlice key, int reduceDims, VectorValueType valueType, ArgSlice value, ArgSlice element, VectorQuantType quantizer, int buildExplorationFactor, ArgSlice attributes, int numLinks, out VectorManagerResult result, out ReadOnlySpan errorMsg);
+
+ ///
+ /// Remove a member from a vector set, if it is present and the key exists.
+ ///
+ GarnetStatus VectorSetRemove(ArgSlice key, ArgSlice element);
+ #endregion
}
///
@@ -1217,7 +1224,7 @@ public interface IGarnetReadApi
///
/// GET
///
- GarnetStatus GET(ref SpanByte key, ref RawStringInput input, ref SpanByteAndMemory output);
+ GarnetStatus GET(ArgSlice key, ref RawStringInput input, ref SpanByteAndMemory output);
///
/// GET
@@ -2026,6 +2033,36 @@ public bool IterateObjectStore(ref TScanFunctions scanFunctions,
#endregion
+ #region Vector Sets
+
+ ///
+ /// Perform a similarity search given a vector and these parameters.
+ ///
+ /// Ids are encoded in as length prefixed blobs of bytes.
+ /// Attributes are encoded in as length prefixed blobs of bytes.
+ ///
+ GarnetStatus VectorSetValueSimilarity(ArgSlice key, VectorValueType valueType, ArgSlice value, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result);
+
+ ///
+ /// Perform a similarity search given an element already in the vector set and these parameters.
+ ///
+ /// Ids are encoded in as length prefixed blobs of bytes.
+ /// Attributes are encoded in as length prefixed blobs of bytes.
+ ///
+ GarnetStatus VectorSetElementSimilarity(ArgSlice key, ArgSlice element, int count, float delta, int searchExplorationFactor, ArgSlice filter, int maxFilteringEffort, bool includeAttributes, ref SpanByteAndMemory outputIds, out VectorIdFormat outputIdFormat, ref SpanByteAndMemory outputDistances, ref SpanByteAndMemory outputAttributes, out VectorManagerResult result);
+
+ ///
+ /// Fetch the embedding of a given element in a Vector set.
+ ///
+ GarnetStatus VectorSetEmbedding(ArgSlice key, ArgSlice element, ref SpanByteAndMemory outputDistances);
+
+ ///
+ /// Fetch the dimensionality of the given Vector Set.
+ ///
+ /// If the Vector Set was created with reduced dimensions, reports the reduced dimensions.
+ ///
+ GarnetStatus VectorSetDimensions(ArgSlice key, out int dimensions);
+ #endregion
}
///
diff --git a/libs/server/ArgSlice/ArgSliceVector.cs b/libs/server/ArgSlice/ArgSliceVector.cs
index 07091e1b130..26e792d4f56 100644
--- a/libs/server/ArgSlice/ArgSliceVector.cs
+++ b/libs/server/ArgSlice/ArgSliceVector.cs
@@ -4,6 +4,8 @@
using System;
using System.Collections;
using System.Collections.Generic;
+using System.Diagnostics;
+using Tsavorite.core;
namespace Garnet.server
{
@@ -11,13 +13,13 @@ namespace Garnet.server
/// Vector of ArgSlices
///
///
- public unsafe class ArgSliceVector(int maxItemNum = 1 << 18) : IEnumerable
+ public unsafe class ArgSliceVector(int maxItemNum = 1 << 18) : IEnumerable
{
ScratchBufferBuilder bufferManager = new();
readonly int maxCount = maxItemNum;
public int Count => items.Count;
public bool IsEmpty => items.Count == 0;
- readonly List items = [];
+ readonly List items = [];
///
/// Try to add ArgSlice
@@ -29,7 +31,32 @@ public bool TryAddItem(Span item)
if (Count + 1 >= maxCount)
return false;
- items.Add(bufferManager.CreateArgSlice(item));
+ var argSlice = bufferManager.CreateArgSlice(item);
+
+ items.Add(argSlice.SpanByte);
+ return true;
+ }
+
+ ///
+ /// Try to add ArgSlice
+ ///
+ ///
+ /// True if it succeeds to add ArgSlice, false if maxCount has been reached.
+ public bool TryAddItem(ulong ns, Span item)
+ {
+ Debug.Assert(ns <= byte.MaxValue, "Only byte-size namespaces supported currently");
+
+ if (Count + 1 >= maxCount)
+ return false;
+
+ var argSlice = bufferManager.CreateArgSlice(item.Length + 1);
+ var sb = argSlice.SpanByte;
+
+ sb.MarkNamespace();
+ sb.SetNamespaceInPayload((byte)ns);
+ item.CopyTo(sb.AsSpan());
+
+ items.Add(sb);
return true;
}
@@ -42,7 +69,7 @@ public void Clear()
bufferManager.Reset();
}
- public IEnumerator GetEnumerator()
+ public IEnumerator GetEnumerator()
{
foreach (var item in items)
yield return item;
diff --git a/libs/server/Cluster/ClusterSlotVerificationInput.cs b/libs/server/Cluster/ClusterSlotVerificationInput.cs
index 8b673189add..0d72b177363 100644
--- a/libs/server/Cluster/ClusterSlotVerificationInput.cs
+++ b/libs/server/Cluster/ClusterSlotVerificationInput.cs
@@ -34,5 +34,12 @@ public struct ClusterSlotVerificationInput
/// Offset of key num if any
///
public int keyNumOffset;
+
+ ///
+ /// If the command being executed modifes a Vector Set.
+ ///
+ /// This requires special handling during migrations.
+ ///
+ public bool isVectorSetWriteCommand;
}
}
\ No newline at end of file
diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs
index 344c88c41e2..f8d854ed409 100644
--- a/libs/server/Cluster/IClusterProvider.cs
+++ b/libs/server/Cluster/IClusterProvider.cs
@@ -12,22 +12,33 @@
namespace Garnet.server
{
+ using BasicContext = BasicContext,
+ SpanByteAllocator>>;
+
using BasicGarnetApi = GarnetApi,
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
+
+ using VectorContext = BasicContext, SpanByteAllocator>>;
///
/// Cluster provider
///
public interface IClusterProvider : IDisposable
{
+ // TODO: I really hate having to pass Vector and Basic contexts here... cleanup
+
///
/// Create cluster session
///
- IClusterSession CreateClusterSession(TransactionManager txnManager, IGarnetAuthenticator authenticator, UserHandle userHandle, GarnetSessionMetrics garnetSessionMetrics, BasicGarnetApi basicGarnetApi, INetworkSender networkSender, ILogger logger = null);
+ IClusterSession CreateClusterSession(TransactionManager txnManager, IGarnetAuthenticator authenticator, UserHandle userHandle, GarnetSessionMetrics garnetSessionMetrics, BasicGarnetApi basicGarnetApi, BasicContext basicContext, VectorContext vectorContext, INetworkSender networkSender, ILogger logger = null);
///
diff --git a/libs/server/Cluster/IClusterSession.cs b/libs/server/Cluster/IClusterSession.cs
index 045d4de959b..af7ceaa5cfd 100644
--- a/libs/server/Cluster/IClusterSession.cs
+++ b/libs/server/Cluster/IClusterSession.cs
@@ -62,7 +62,7 @@ public interface IClusterSession
///
/// Process cluster commands
///
- unsafe void ProcessClusterCommands(RespCommand command, ref SessionParseState parseState, ref byte* dcurr, ref byte* dend);
+ unsafe void ProcessClusterCommands(RespCommand command, VectorManager vectorManager, ref SessionParseState parseState, ref byte* dcurr, ref byte* dend);
///
/// Reset cached slot verification result
@@ -77,7 +77,7 @@ public interface IClusterSession
///
///
///
- bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking);
+ bool NetworkIterativeSlotVerify(ArgSlice keySlice, bool readOnly, byte SessionAsking, bool isVectorSetWriteCommand);
///
/// Write cached slot verification message to output
@@ -88,7 +88,7 @@ public interface IClusterSession
///
/// Key array slot verify (write result to network)
///
- unsafe bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte SessionAsking, ref byte* dcurr, ref byte* dend, int count = -1);
+ unsafe bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, byte SessionAsking, bool isVectorSetWriteCommand, ref byte* dcurr, ref byte* dend, int count = -1);
///
/// Array slot verify (write result to network)
diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs
index 2700eaa088c..04c823a8727 100644
--- a/libs/server/Databases/DatabaseManagerBase.cs
+++ b/libs/server/Databases/DatabaseManagerBase.cs
@@ -414,7 +414,7 @@ protected void ExecuteObjectCollection(GarnetDatabase db, ILogger logger = null)
{
var scratchBufferManager = new ScratchBufferBuilder();
db.ObjectStoreCollectionDbStorageSession =
- new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
+ new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, db.VectorManager, Logger);
}
ExecuteHashCollect(db.ObjectStoreCollectionDbStorageSession);
@@ -722,7 +722,7 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession)
if (db.MainStoreExpiredKeyDeletionDbStorageSession == null)
{
var scratchBufferManager = new ScratchBufferBuilder();
- db.MainStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
+ db.MainStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, db.VectorManager, Logger);
}
var scanFrom = StoreWrapper.store.Log.ReadOnlyAddress;
@@ -738,7 +738,7 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession)
if (db.ObjectStoreExpiredKeyDeletionDbStorageSession == null)
{
var scratchBufferManager = new ScratchBufferBuilder();
- db.ObjectStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
+ db.ObjectStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, db.VectorManager, Logger);
}
var scanFrom = StoreWrapper.objectStore.Log.ReadOnlyAddress;
@@ -778,7 +778,7 @@ private HybridLogScanMetrics CollectHybridLogStats>(sessionFunctions);
diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs
index 6d5855f5dd5..14fe8f4d685 100644
--- a/libs/server/Databases/MultiDatabaseManager.cs
+++ b/libs/server/Databases/MultiDatabaseManager.cs
@@ -147,6 +147,9 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover
if (StoreWrapper.serverOptions.FailOnRecoveryError)
throw new GarnetException("Main store and object store checkpoint versions do not match");
}
+
+ // Once everything is setup, initialize the VectorManager
+ db.VectorManager.Initialize();
}
}
@@ -712,7 +715,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto
throw new GarnetException($"Database with ID {dbId} was not found.");
return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper.customCommandManager, null, db.ObjectStoreSizeTracker,
- StoreWrapper.GarnetObjectSerializer, respProtocolVersion);
+ StoreWrapper.GarnetObjectSerializer, db.VectorManager, respProtocolVersion);
}
///
diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs
index 605262dd098..15a3423f88c 100644
--- a/libs/server/Databases/SingleDatabaseManager.cs
+++ b/libs/server/Databases/SingleDatabaseManager.cs
@@ -111,6 +111,9 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover
if (StoreWrapper.serverOptions.FailOnRecoveryError)
throw new GarnetException("Main store and object store checkpoint versions do not match");
}
+
+ // Once everything is setup, initialize the VectorManager
+ defaultDatabase.VectorManager.Initialize();
}
///
@@ -391,7 +394,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto
ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0);
return new(AppendOnlyFile, VersionMap, StoreWrapper.customCommandManager, null, ObjectStoreSizeTracker,
- StoreWrapper.GarnetObjectSerializer, respProtocolVersion);
+ StoreWrapper.GarnetObjectSerializer, DefaultDatabase.VectorManager, respProtocolVersion);
}
private async Task TryPauseCheckpointsContinuousAsync(int dbId,
diff --git a/libs/server/Garnet.server.csproj b/libs/server/Garnet.server.csproj
index 2c351e80f45..dc679f37e8f 100644
--- a/libs/server/Garnet.server.csproj
+++ b/libs/server/Garnet.server.csproj
@@ -22,6 +22,7 @@
+
\ No newline at end of file
diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs
index 41eb4784f6d..ef3788c7e85 100644
--- a/libs/server/GarnetDatabase.cs
+++ b/libs/server/GarnetDatabase.cs
@@ -100,6 +100,14 @@ public class GarnetDatabase : IDisposable
///
public SingleWriterMultiReaderLock CheckpointingLock;
+ ///
+ /// Per-DB VectorManager
+ ///
+ /// Contexts, metadata, and associated namespaces are DB-specific, and meaningless
+ /// outside of the container DB.
+ ///
+ public readonly VectorManager VectorManager;
+
///
/// Storage session intended for store-wide object collection operations
///
@@ -124,7 +132,7 @@ public GarnetDatabase(int id, TsavoriteKV objectStore,
LightEpoch epoch, StateMachineDriver stateMachineDriver,
CacheSizeTracker objectStoreSizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile,
- bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut) : this()
+ bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut, VectorManager vectorManager) : this()
{
Id = id;
MainStore = mainStore;
@@ -136,6 +144,7 @@ public GarnetDatabase(int id, TsavoriteKV
+ /// Header for Garnet Main Store inputs but for Vector element r/w/d ops
+ ///
+ public struct VectorInput : IStoreInput
+ {
+ public int SerializedLength => throw new NotImplementedException();
+
+ public int ReadDesiredSize { get; set; }
+
+ public int WriteDesiredSize { get; set; }
+
+ public int Index { get; set; }
+ public nint CallbackContext { get; set; }
+ public nint Callback { get; set; }
+
+ public VectorInput()
+ {
+ }
+
+ public unsafe int CopyTo(byte* dest, int length) => throw new NotImplementedException();
+ public unsafe int DeserializeFrom(byte* src) => throw new NotImplementedException();
+ }
}
\ No newline at end of file
diff --git a/libs/server/Resp/AdminCommands.cs b/libs/server/Resp/AdminCommands.cs
index 73851314355..fa134a1498f 100644
--- a/libs/server/Resp/AdminCommands.cs
+++ b/libs/server/Resp/AdminCommands.cs
@@ -703,7 +703,7 @@ private bool NetworkProcessClusterCommand(RespCommand command)
return AbortWithErrorMessage(CmdStrings.RESP_ERR_GENERIC_CLUSTER_DISABLED);
}
- clusterSession.ProcessClusterCommands(command, ref parseState, ref dcurr, ref dend);
+ clusterSession.ProcessClusterCommands(command, storageSession.vectorManager, ref parseState, ref dcurr, ref dend);
return true;
}
diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs
index 6cc37408b4a..838e65d3b21 100644
--- a/libs/server/Resp/BasicCommands.cs
+++ b/libs/server/Resp/BasicCommands.cs
@@ -31,12 +31,13 @@ bool NetworkGET(ref TGarnetApi storageApi)
RawStringInput input = default;
- var key = parseState.GetArgSliceByRef(0).SpanByte;
+ ref var key = ref parseState.GetArgSliceByRef(0);
var o = new SpanByteAndMemory(dcurr, (int)(dend - dcurr));
- var status = storageApi.GET(ref key, ref input, ref o);
+ var status = storageApi.GET(key, ref input, ref o);
switch (status)
{
+ case GarnetStatus.WRONGTYPE:
case GarnetStatus.OK:
if (!o.IsSpanByte)
SendAndReset(o.Memory, o.Length);
@@ -278,10 +279,10 @@ private bool NetworkSET(ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
Debug.Assert(parseState.Count == 2);
- var key = parseState.GetArgSliceByRef(0).SpanByte;
- var value = parseState.GetArgSliceByRef(1).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
+ var value = parseState.GetArgSliceByRef(1);
- storageApi.SET(ref key, ref value);
+ storageApi.SET(key, value);
while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend))
SendAndReset();
@@ -296,9 +297,9 @@ private bool NetworkGETSET(ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
Debug.Assert(parseState.Count == 2);
- var key = parseState.GetArgSliceByRef(0).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
- return NetworkSET_Conditional(RespCommand.SET, 0, ref key, true,
+ return NetworkSET_Conditional(RespCommand.SET, 0, key, true,
false, false, ref storageApi);
}
@@ -377,7 +378,7 @@ private bool NetworkGetRange(ref TGarnetApi storageApi)
private bool NetworkSETEX(bool highPrecision, ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
- var key = parseState.GetArgSliceByRef(0).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
// Validate expiry
if (!parseState.TryGetInt(1, out var expiry))
@@ -398,7 +399,7 @@ private bool NetworkSETEX(bool highPrecision, ref TGarnetApi storage
var sbVal = parseState.GetArgSliceByRef(2).SpanByte;
var input = new RawStringInput(RespCommand.SETEX, 0, valMetadata);
- _ = storageApi.SET(ref key, ref input, ref sbVal);
+ _ = storageApi.SET(key, ref input, ref sbVal);
while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend))
SendAndReset();
@@ -418,10 +419,9 @@ private bool NetworkSETNX(bool highPrecision, ref TGarnetApi storage
}
var key = parseState.GetArgSliceByRef(0);
- var sbKey = key.SpanByte;
var input = new RawStringInput(RespCommand.SETEXNX, ref parseState, startIdx: 1);
- var status = storageApi.SET_Conditional(ref sbKey, ref input);
+ var status = storageApi.SET_Conditional(key, ref input);
// The status returned for SETNX as NOTFOUND is the expected status in the happy path
var retVal = status == GarnetStatus.NOTFOUND ? 1 : 0;
@@ -573,14 +573,14 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi)
{
case ExistOptions.None:
return getValue || withEtag
- ? NetworkSET_Conditional(RespCommand.SET, expiry, ref sbKey, getValue,
+ ? NetworkSET_Conditional(RespCommand.SET, expiry, key, getValue,
isHighPrecision, withEtag, ref storageApi)
- : NetworkSET_EX(RespCommand.SET, expOption, expiry, ref sbKey, ref sbVal, ref storageApi); // Can perform a blind update
+ : NetworkSET_EX(RespCommand.SET, expOption, expiry, key, ref sbVal, ref storageApi); // Can perform a blind update
case ExistOptions.XX:
- return NetworkSET_Conditional(RespCommand.SETEXXX, expiry, ref sbKey,
+ return NetworkSET_Conditional(RespCommand.SETEXXX, expiry, key,
getValue, isHighPrecision, withEtag, ref storageApi);
case ExistOptions.NX:
- return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, ref sbKey,
+ return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, key,
getValue, isHighPrecision, withEtag, ref storageApi);
}
break;
@@ -590,13 +590,13 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi)
{
case ExistOptions.None:
// We can never perform a blind update due to KEEPTTL
- return NetworkSET_Conditional(RespCommand.SETKEEPTTL, expiry, ref sbKey
+ return NetworkSET_Conditional(RespCommand.SETKEEPTTL, expiry, key
, getValue, highPrecision: false, withEtag, ref storageApi);
case ExistOptions.XX:
- return NetworkSET_Conditional(RespCommand.SETKEEPTTLXX, expiry, ref sbKey,
+ return NetworkSET_Conditional(RespCommand.SETKEEPTTLXX, expiry, key,
getValue, highPrecision: false, withEtag, ref storageApi);
case ExistOptions.NX:
- return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, ref sbKey,
+ return NetworkSET_Conditional(RespCommand.SETEXNX, expiry, key,
getValue, highPrecision: false, withEtag, ref storageApi);
}
break;
@@ -608,7 +608,7 @@ private bool NetworkSETEXNX(ref TGarnetApi storageApi)
}
private unsafe bool NetworkSET_EX(RespCommand cmd, ExpirationOption expOption, int expiry,
- ref SpanByte key, ref SpanByte val, ref TGarnetApi storageApi)
+ ArgSlice key, ref SpanByte val, ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
Debug.Assert(cmd == RespCommand.SET);
@@ -621,14 +621,14 @@ private unsafe bool NetworkSET_EX(RespCommand cmd, ExpirationOption
var input = new RawStringInput(cmd, 0, valMetadata);
- storageApi.SET(ref key, ref input, ref val);
+ storageApi.SET(key, ref input, ref val);
while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend))
SendAndReset();
return true;
}
- private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref SpanByte key, bool getValue, bool highPrecision, bool withEtag, ref TGarnetApi storageApi)
+ private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ArgSlice key, bool getValue, bool highPrecision, bool withEtag, ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
var inputArg = expiry == 0
@@ -645,7 +645,7 @@ private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref
// the following debug assertion is the catch any edge case leading to SETIFMATCH, or SETIFGREATER skipping the above block
Debug.Assert(cmd is not (RespCommand.SETIFMATCH or RespCommand.SETIFGREATER), "SETIFMATCH should have gone though pointing to right output variable");
- var status = storageApi.SET_Conditional(ref key, ref input);
+ var status = storageApi.SET_Conditional(key, ref input);
// KEEPTTL without flags doesn't care whether it was found or not.
if (cmd == RespCommand.SETKEEPTTL)
@@ -684,7 +684,7 @@ private bool NetworkSET_Conditional(RespCommand cmd, int expiry, ref
// anything with getValue or withEtag always writes to the buffer in the happy path
SpanByteAndMemory outputBuffer = new SpanByteAndMemory(dcurr, (int)(dend - dcurr));
- GarnetStatus status = storageApi.SET_Conditional(ref key, ref input, ref outputBuffer);
+ GarnetStatus status = storageApi.SET_Conditional(key, ref input, ref outputBuffer);
// The data will be on the buffer either when we know the response is ok or when the withEtag flag is set.
bool ok = status != GarnetStatus.NOTFOUND || withEtag;
diff --git a/libs/server/Resp/BasicEtagCommands.cs b/libs/server/Resp/BasicEtagCommands.cs
index 59ef098eaa7..2fee440918d 100644
--- a/libs/server/Resp/BasicEtagCommands.cs
+++ b/libs/server/Resp/BasicEtagCommands.cs
@@ -22,10 +22,10 @@ private bool NetworkGETWITHETAG(ref TGarnetApi storageApi)
{
Debug.Assert(parseState.Count == 1);
- var key = parseState.GetArgSliceByRef(0).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
var input = new RawStringInput(RespCommand.GETWITHETAG);
var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr));
- var status = storageApi.GET(ref key, ref input, ref output);
+ var status = storageApi.GET(key, ref input, ref output);
switch (status)
{
@@ -53,10 +53,10 @@ private bool NetworkGETIFNOTMATCH(ref TGarnetApi storageApi)
{
Debug.Assert(parseState.Count == 2);
- var key = parseState.GetArgSliceByRef(0).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
var input = new RawStringInput(RespCommand.GETIFNOTMATCH, ref parseState, startIdx: 1);
var output = new SpanByteAndMemory(dcurr, (int)(dend - dcurr));
- var status = storageApi.GET(ref key, ref input, ref output);
+ var status = storageApi.GET(key, ref input, ref output);
switch (status)
{
@@ -213,9 +213,9 @@ private bool NetworkSetETagConditional(RespCommand cmd, ref TGarnetA
return true;
}
- SpanByte key = parseState.GetArgSliceByRef(0).SpanByte;
+ var key = parseState.GetArgSliceByRef(0);
- NetworkSET_Conditional(cmd, expiry, ref key, getValue: !noGet, highPrecision: expOption == ExpirationOption.PX, withEtag: true, ref storageApi);
+ NetworkSET_Conditional(cmd, expiry, key, getValue: !noGet, highPrecision: expOption == ExpirationOption.PX, withEtag: true, ref storageApi);
return true;
}
diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs
index cd3263aa808..e8c5ba5fb9e 100644
--- a/libs/server/Resp/CmdStrings.cs
+++ b/libs/server/Resp/CmdStrings.cs
@@ -440,6 +440,7 @@ static partial class CmdStrings
public static ReadOnlySpan publish => "PUBLISH"u8;
public static ReadOnlySpan spublish => "SPUBLISH"u8;
public static ReadOnlySpan mtasks => "MTASKS"u8;
+ public static ReadOnlySpan reserve => "RESERVE"u8;
public static ReadOnlySpan aofsync => "AOFSYNC"u8;
public static ReadOnlySpan appendlog => "APPENDLOG"u8;
public static ReadOnlySpan attach_sync => "ATTACH_SYNC"u8;
diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs
index 0e52d40d9c1..1eed9e96553 100644
--- a/libs/server/Resp/GarnetDatabaseSession.cs
+++ b/libs/server/Resp/GarnetDatabaseSession.cs
@@ -8,13 +8,19 @@ namespace Garnet.server
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
using LockableGarnetApi = GarnetApi,
SpanByteAllocator>>,
LockableContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ LockableContext,
+ SpanByteAllocator>>>;
///
/// Represents a logical database session in Garnet
diff --git a/libs/server/Resp/KeyAdminCommands.cs b/libs/server/Resp/KeyAdminCommands.cs
index 812617a3a57..1e9e18efefe 100644
--- a/libs/server/Resp/KeyAdminCommands.cs
+++ b/libs/server/Resp/KeyAdminCommands.cs
@@ -99,8 +99,6 @@ bool NetworkRESTORE(ref TGarnetApi storageApi)
var valArgSlice = scratchBufferBuilder.CreateArgSlice(val);
- var sbKey = key.SpanByte;
-
parseState.InitializeWithArgument(valArgSlice);
RawStringInput input;
@@ -114,7 +112,7 @@ bool NetworkRESTORE(ref TGarnetApi storageApi)
input = new RawStringInput(RespCommand.SETEXNX, ref parseState);
}
- var status = storageApi.SET_Conditional(ref sbKey, ref input);
+ var status = storageApi.SET_Conditional(key, ref input);
if (status is GarnetStatus.NOTFOUND)
{
diff --git a/libs/server/Resp/LocalServerSession.cs b/libs/server/Resp/LocalServerSession.cs
index b3283504041..3bf4a4ca1c5 100644
--- a/libs/server/Resp/LocalServerSession.cs
+++ b/libs/server/Resp/LocalServerSession.cs
@@ -2,6 +2,7 @@
// Licensed under the MIT license.
using System;
+using System.Diagnostics;
using Microsoft.Extensions.Logging;
using Tsavorite.core;
@@ -12,7 +13,10 @@ namespace Garnet.server
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
///
/// Local server session
@@ -47,8 +51,11 @@ public LocalServerSession(StoreWrapper storeWrapper)
// Initialize session-local scratch buffer of size 64 bytes, used for constructing arguments in GarnetApi
this.scratchBufferBuilder = new ScratchBufferBuilder();
+ var dbRes = storeWrapper.TryGetOrAddDatabase(0, out var database, out _);
+ Debug.Assert(dbRes, "Should always be able to get DB 0");
+
// Create storage session and API
- this.storageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId: 0, logger);
+ this.storageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId: 0, database.VectorManager, logger);
this.BasicGarnetApi = new BasicGarnetApi(storageSession, storageSession.basicContext, storageSession.objectStoreBasicContext);
}
diff --git a/libs/server/Resp/Parser/ParseUtils.cs b/libs/server/Resp/Parser/ParseUtils.cs
index 14d6e0f5edc..02e9a2c41ca 100644
--- a/libs/server/Resp/Parser/ParseUtils.cs
+++ b/libs/server/Resp/Parser/ParseUtils.cs
@@ -130,6 +130,44 @@ public static bool TryReadDouble(ref ArgSlice slice, out double number, bool can
return canBeInfinite && RespReadUtils.TryReadInfinity(sbNumber, out number);
}
+ ///
+ /// Read a signed 32-bit float from a given ArgSlice.
+ ///
+ /// Source
+ /// Allow reading an infinity
+ ///
+ /// Parsed double
+ ///
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public static float ReadFloat(ref ArgSlice slice, bool canBeInfinite)
+ {
+ if (!TryReadFloat(ref slice, out var number, canBeInfinite))
+ {
+ RespParsingException.ThrowNotANumber(slice.ptr, slice.length);
+ }
+ return number;
+ }
+
+ ///
+ /// Try to read a signed 32-bit float from a given ArgSlice.
+ ///
+ /// Source
+ /// Result
+ /// Allow reading an infinity
+ ///
+ /// True if float parsed successfully
+ ///
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public static bool TryReadFloat(ref ArgSlice slice, out float number, bool canBeInfinite)
+ {
+ var sbNumber = slice.ReadOnlySpan;
+ if (Utf8Parser.TryParse(sbNumber, out number, out var bytesConsumed) &&
+ bytesConsumed == sbNumber.Length)
+ return true;
+
+ return canBeInfinite && RespReadUtils.TryReadInfinity(sbNumber, out number);
+ }
+
///
/// Read an ASCII string from a given ArgSlice.
///
diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs
index cc81121b1df..c0f0d906cdc 100644
--- a/libs/server/Resp/Parser/RespCommand.cs
+++ b/libs/server/Resp/Parser/RespCommand.cs
@@ -81,6 +81,15 @@ public enum RespCommand : ushort
SUNION,
TTL,
TYPE,
+ VCARD,
+ VDIM,
+ VEMB,
+ VGETATTR,
+ VINFO,
+ VISMEMBER,
+ VLINKS,
+ VRANDMEMBER,
+ VSIM,
WATCH,
WATCHMS,
WATCHOS,
@@ -195,6 +204,9 @@ public enum RespCommand : ushort
SUNIONSTORE,
SWAPDB,
UNLINK,
+ VADD,
+ VREM,
+ VSETATTR,
ZADD,
ZCOLLECT,
ZDIFFSTORE,
@@ -374,6 +386,7 @@ public enum RespCommand : ushort
CLUSTER_SPUBLISH,
CLUSTER_REPLICAS,
CLUSTER_REPLICATE,
+ CLUSTER_RESERVE,
CLUSTER_RESET,
CLUSTER_SEND_CKPT_FILE_SEGMENT,
CLUSTER_SEND_CKPT_METADATA,
@@ -627,6 +640,12 @@ public static bool IsClusterSubCommand(this RespCommand cmd)
bool inRange = test <= (RespCommand.CLUSTER_SYNC - RespCommand.CLUSTER_ADDSLOTS);
return inRange;
}
+
+ ///
+ /// Returns true if this command can operate on a Vector Set.
+ ///
+ public static bool IsLegalOnVectorSet(this RespCommand cmd)
+ => cmd is RespCommand.DEL or RespCommand.TYPE or RespCommand.DEBUG or RespCommand.VADD or RespCommand.VCARD or RespCommand.VDIM or RespCommand.VEMB or RespCommand.VGETATTR or RespCommand.VINFO or server.RespCommand.VISMEMBER or RespCommand.VLINKS or RespCommand.VRANDMEMBER or RespCommand.VREM or RespCommand.VSETATTR or RespCommand.VSIM;
}
///
@@ -961,6 +980,29 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
}
break;
+ case 'V':
+ if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nVADD\r\n"u8))
+ {
+ return RespCommand.VADD;
+ }
+ else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nVDIM\r\n"u8))
+ {
+ return RespCommand.VDIM;
+ }
+ else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nVEMB\r\n"u8))
+ {
+ return RespCommand.VEMB;
+ }
+ else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nVREM\r\n"u8))
+ {
+ return RespCommand.VREM;
+ }
+ else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nVSIM\r\n"u8))
+ {
+ return RespCommand.VSIM;
+ }
+ break;
+
case 'Z':
if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("\r\nZADD\r\n"u8))
{
@@ -1141,6 +1183,17 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
}
break;
+ case 'V':
+ if (*(ulong*)(ptr + 3) == MemoryMarshal.Read("\nVCARD\r\n"u8))
+ {
+ return RespCommand.VCARD;
+ }
+ else if (*(ulong*)(ptr + 3) == MemoryMarshal.Read("\nVINFO\r\n"u8))
+ {
+ return RespCommand.VINFO;
+ }
+ break;
+
case 'W':
if (*(ulong*)(ptr + 3) == MemoryMarshal.Read("\nWATCH\r\n"u8))
{
@@ -1335,6 +1388,13 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
}
break;
+ case 'V':
+ if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("VLINKS\r\n"u8))
+ {
+ return RespCommand.VLINKS;
+ }
+ break;
+
case 'Z':
if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("ZCOUNT\r\n"u8))
{
@@ -1510,6 +1570,14 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
{
return RespCommand.SPUBLISH;
}
+ else if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("VGETATTR"u8) && *(ushort*)(ptr + 12) == MemoryMarshal.Read("\r\n"u8))
+ {
+ return RespCommand.VGETATTR;
+ }
+ else if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("VSETATTR"u8) && *(ushort*)(ptr + 12) == MemoryMarshal.Read("\r\n"u8))
+ {
+ return RespCommand.VSETATTR;
+ }
break;
case 9:
if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("SUBSCRIB"u8) && *(uint*)(ptr + 11) == MemoryMarshal.Read("BE\r\n"u8))
@@ -1548,6 +1616,10 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
{
return RespCommand.ZEXPIREAT;
}
+ else if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("VISMEMBE"u8) && *(uint*)(ptr + 11) == MemoryMarshal.Read("ER\r\n"u8))
+ {
+ return RespCommand.VISMEMBER;
+ }
break;
case 10:
if (*(ulong*)(ptr + 4) == MemoryMarshal.Read("SSUBSCRI"u8) && *(uint*)(ptr + 11) == MemoryMarshal.Read("BE\r\n"u8))
@@ -1684,6 +1756,10 @@ private RespCommand FastParseArrayCommand(ref int count, ref ReadOnlySpan
{
return RespCommand.ZEXPIRETIME;
}
+ else if (*(ulong*)(ptr + 2) == MemoryMarshal.Read("1\r\nVRAND"u8) && *(ulong*)(ptr + 10) == MemoryMarshal.Read("MEMBER\r\n"u8))
+ {
+ return RespCommand.VRANDMEMBER;
+ }
break;
case 12:
@@ -2201,6 +2277,10 @@ private RespCommand SlowParseCommand(ReadOnlySpan command, ref int count,
{
return RespCommand.CLUSTER_MIGRATE;
}
+ else if (subCommand.SequenceEqual(CmdStrings.reserve))
+ {
+ return RespCommand.CLUSTER_RESERVE;
+ }
else if (subCommand.SequenceEqual(CmdStrings.mtasks))
{
return RespCommand.CLUSTER_MTASKS;
diff --git a/libs/server/Resp/Parser/SessionParseState.cs b/libs/server/Resp/Parser/SessionParseState.cs
index e0e523c7ea2..358b37b14fc 100644
--- a/libs/server/Resp/Parser/SessionParseState.cs
+++ b/libs/server/Resp/Parser/SessionParseState.cs
@@ -163,18 +163,19 @@ public void InitializeWithArguments(ArgSlice arg1, ArgSlice arg2, ArgSlice arg3,
}
///
- /// Initialize the parse state with a given set of arguments
+ /// Expand (if necessary) capacity of , preserving contents.
///
- /// Set of arguments to initialize buffer with
- [MethodImpl(MethodImplOptions.AggressiveInlining)]
- public void InitializeWithArguments(ArgSlice[] args)
+ public void EnsureCapacity(int count)
{
- Initialize(args.Length);
-
- for (var i = 0; i < args.Length; i++)
+ if (count <= Count)
{
- *(bufferPtr + i) = args[i];
+ return;
}
+
+ var oldBuffer = rootBuffer;
+ Initialize(count);
+
+ oldBuffer?.AsSpan().CopyTo(rootBuffer);
}
///
@@ -432,6 +433,28 @@ public bool TryGetDouble(int i, out double value, bool canBeInfinite = true)
return ParseUtils.TryReadDouble(ref Unsafe.AsRef(bufferPtr + i), out value, canBeInfinite);
}
+ ///
+ /// Get float argument at the given index
+ ///
+ /// True if double parsed successfully
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public float GetFloat(int i, bool canBeInfinite = true)
+ {
+ Debug.Assert(i < Count);
+ return ParseUtils.ReadFloat(ref Unsafe.AsRef(bufferPtr + i), canBeInfinite);
+ }
+
+ ///
+ /// Try to get double argument at the given index
+ ///
+ /// True if double parsed successfully
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public bool TryGetFloat(int i, out float value, bool canBeInfinite = true)
+ {
+ Debug.Assert(i < Count);
+ return ParseUtils.TryReadFloat(ref Unsafe.AsRef(bufferPtr + i), out value, canBeInfinite);
+ }
+
///
/// Get ASCII string argument at the given index
///
diff --git a/libs/server/Resp/RespCommandDocs.cs b/libs/server/Resp/RespCommandDocs.cs
index f6adceaecf0..b58578f7371 100644
--- a/libs/server/Resp/RespCommandDocs.cs
+++ b/libs/server/Resp/RespCommandDocs.cs
@@ -330,6 +330,8 @@ public enum RespCommandGroup : byte
String,
[Description("transactions")]
Transactions,
+ [Description("vector")]
+ Vector
}
///
diff --git a/libs/server/Resp/RespCommandInfoFlags.cs b/libs/server/Resp/RespCommandInfoFlags.cs
index e4f391a8613..bfe03845bf7 100644
--- a/libs/server/Resp/RespCommandInfoFlags.cs
+++ b/libs/server/Resp/RespCommandInfoFlags.cs
@@ -55,6 +55,8 @@ public enum RespCommandFlags
Write = 1 << 19,
[Description("allow_busy")]
AllowBusy = 1 << 20,
+ [Description("module")]
+ Module = 1 << 21,
}
///
@@ -110,6 +112,8 @@ public enum RespAclCategories
Garnet = 1 << 21,
[Description("custom")]
Custom = 1 << 22,
+ [Description("vector")]
+ Vector = 1 << 23,
[Description("all")]
All = (Custom << 1) - 1,
}
diff --git a/libs/server/Resp/RespServerSession.cs b/libs/server/Resp/RespServerSession.cs
index 854def816d9..8d8894e89b1 100644
--- a/libs/server/Resp/RespServerSession.cs
+++ b/libs/server/Resp/RespServerSession.cs
@@ -25,13 +25,19 @@ namespace Garnet.server
SpanByteAllocator>>,
BasicContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ BasicContext,
+ SpanByteAllocator>>>;
using LockableGarnetApi = GarnetApi,
SpanByteAllocator>>,
LockableContext>,
- GenericAllocator>>>>;
+ GenericAllocator>>>,
+ LockableContext,
+ SpanByteAllocator>>>;
///
/// RESP server session
@@ -283,7 +289,8 @@ public RespServerSession(
this.AuthenticateUser(Encoding.ASCII.GetBytes(this.storeWrapper.accessControlList.GetDefaultUserHandle().User.Name));
var cp = clusterProvider ?? storeWrapper.clusterProvider;
- clusterSession = cp?.CreateClusterSession(txnManager, this._authenticator, this._userHandle, sessionMetrics, basicGarnetApi, networkSender, logger);
+
+ clusterSession = cp?.CreateClusterSession(txnManager, this._authenticator, this._userHandle, sessionMetrics, basicGarnetApi, storageSession.basicContext, storageSession.vectorContext, networkSender, logger);
clusterSession?.SetUserHandle(this._userHandle);
sessionScriptCache?.SetUserHandle(this._userHandle);
@@ -946,6 +953,20 @@ private bool ProcessArrayCommands(RespCommand cmd, ref TGarnetApi st
RespCommand.SUNIONSTORE => SetUnionStore(ref storageApi),
RespCommand.SDIFF => SetDiff(ref storageApi),
RespCommand.SDIFFSTORE => SetDiffStore(ref storageApi),
+ // Vector Commands
+ RespCommand.VADD => NetworkVADD(ref storageApi),
+ RespCommand.VCARD => NetworkVCARD(ref storageApi),
+ RespCommand.VDIM => NetworkVDIM(ref storageApi),
+ RespCommand.VEMB => NetworkVEMB(ref storageApi),
+ RespCommand.VGETATTR => NetworkVGETATTR(ref storageApi),
+ RespCommand.VINFO => NetworkVINFO(ref storageApi),
+ RespCommand.VISMEMBER => NetworkVISMEMBER(ref storageApi),
+ RespCommand.VLINKS => NetworkVLINKS(ref storageApi),
+ RespCommand.VRANDMEMBER => NetworkVRANDMEMBER(ref storageApi),
+ RespCommand.VREM => NetworkVREM(ref storageApi),
+ RespCommand.VSETATTR => NetworkVSETATTR(ref storageApi),
+ RespCommand.VSIM => NetworkVSIM(ref storageApi),
+ // Everything else
_ => ProcessOtherCommands(cmd, ref storageApi)
};
return success;
@@ -1332,7 +1353,7 @@ private void Send(byte* d)
if ((int)(dcurr - d) > 0)
{
- // Debug.WriteLine("SEND: [" + Encoding.UTF8.GetString(new Span(d, (int)(dcurr - d))).Replace("\n", "|").Replace("\r", "!") + "]");
+ //Debug.WriteLine("SEND: [" + Encoding.UTF8.GetString(new Span(d, (int)(dcurr - d))).Replace("\n", "|").Replace("\r", "!") + "]");
if (waitForAofBlocking)
{
var task = storeWrapper.WaitForCommitAsync();
@@ -1496,7 +1517,10 @@ private GarnetDatabaseSession TryGetOrSetDatabaseSession(int dbId, out bool succ
/// New database session
private GarnetDatabaseSession CreateDatabaseSession(int dbId)
{
- var dbStorageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId, logger, respProtocolVersion);
+ var dbRes = storeWrapper.TryGetOrAddDatabase(dbId, out var database, out _);
+ Debug.Assert(dbRes, "Should always find database if we're switching to it");
+
+ var dbStorageSession = new StorageSession(storeWrapper, scratchBufferBuilder, sessionMetrics, LatencyMetrics, dbId, database.VectorManager, logger, respProtocolVersion);
var dbGarnetApi = new BasicGarnetApi(dbStorageSession, dbStorageSession.basicContext, dbStorageSession.objectStoreBasicContext);
var dbLockableGarnetApi = new LockableGarnetApi(dbStorageSession, dbStorageSession.lockableContext, dbStorageSession.objectStoreLockableContext);
diff --git a/libs/server/Resp/RespServerSessionSlotVerify.cs b/libs/server/Resp/RespServerSessionSlotVerify.cs
index 9de8ee1c18d..39179c979f5 100644
--- a/libs/server/Resp/RespServerSessionSlotVerify.cs
+++ b/libs/server/Resp/RespServerSessionSlotVerify.cs
@@ -17,9 +17,10 @@ internal sealed unsafe partial class RespServerSession : ServerSessionBase
/// Array of key ArgSlice
/// Whether caller is going to perform a readonly or read/write operation
/// Key count if different than keys array length
+ /// Whether the executing command performs a write against a Vector Set.
/// True when ownership is verified, false otherwise
- bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, int count = -1)
- => clusterSession != null && clusterSession.NetworkKeyArraySlotVerify(keys, readOnly, SessionAsking, ref dcurr, ref dend, count);
+ bool NetworkKeyArraySlotVerify(Span keys, bool readOnly, bool isVectorSetWriteCommand, int count = -1)
+ => clusterSession != null && clusterSession.NetworkKeyArraySlotVerify(keys, readOnly, SessionAsking, isVectorSetWriteCommand, ref dcurr, ref dend, count);
bool CanServeSlot(RespCommand cmd)
{
@@ -43,6 +44,7 @@ bool CanServeSlot(RespCommand cmd)
storeWrapper.clusterProvider.ExtractKeySpecs(commandInfo, cmd, ref parseState, ref csvi);
csvi.readOnly = cmd.IsReadOnly();
csvi.sessionAsking = SessionAsking;
+ csvi.isVectorSetWriteCommand = cmd is RespCommand.VADD or RespCommand.VREM or RespCommand.VSETATTR;
return !clusterSession.NetworkMultiKeySlotVerify(ref parseState, ref csvi, ref dcurr, ref dend);
}
}
diff --git a/libs/server/Resp/Vector/DiskANNService.cs b/libs/server/Resp/Vector/DiskANNService.cs
new file mode 100644
index 00000000000..8a178af5eff
--- /dev/null
+++ b/libs/server/Resp/Vector/DiskANNService.cs
@@ -0,0 +1,319 @@
+using System;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+namespace Garnet.server
+{
+ internal sealed unsafe class DiskANNService
+ {
+ // Term types.
+ internal const byte FullVector = 0;
+ private const byte NeighborList = 1;
+ private const byte QuantizedVector = 2;
+ internal const byte Attributes = 3;
+
+ public nint CreateIndex(
+ ulong context,
+ uint dimensions,
+ uint reduceDims,
+ VectorQuantType quantType,
+ uint buildExplorationFactor,
+ uint numLinks,
+ delegate* unmanaged[Cdecl] readCallback,
+ delegate* unmanaged[Cdecl] writeCallback,
+ delegate* unmanaged[Cdecl] deleteCallback,
+ delegate* unmanaged[Cdecl] readModifyWriteCallback
+ )
+ {
+ unsafe
+ {
+ return NativeDiskANNMethods.create_index(context, dimensions, reduceDims, quantType, buildExplorationFactor, numLinks, (nint)readCallback, (nint)writeCallback, (nint)deleteCallback, (nint)readModifyWriteCallback);
+ }
+ }
+
+ public nint RecreateIndex(
+ ulong context,
+ uint dimensions,
+ uint reduceDims,
+ VectorQuantType quantType,
+ uint buildExplorationFactor,
+ uint numLinks,
+ delegate* unmanaged[Cdecl] readCallback,
+ delegate* unmanaged[Cdecl] writeCallback,
+ delegate* unmanaged[Cdecl] deleteCallback,
+ delegate* unmanaged[Cdecl] readModifyWriteCallback
+ )
+ => CreateIndex(context, dimensions, reduceDims, quantType, buildExplorationFactor, numLinks, readCallback, writeCallback, deleteCallback, readModifyWriteCallback);
+
+ public void DropIndex(ulong context, nint index)
+ {
+ NativeDiskANNMethods.drop_index(context, index);
+ }
+
+ public bool Insert(ulong context, nint index, ReadOnlySpan id, VectorValueType vectorType, ReadOnlySpan vector, ReadOnlySpan attributes)
+ {
+ var id_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(id));
+ var id_len = id.Length;
+
+ var vector_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(vector));
+ int vector_len;
+
+ if (vectorType == VectorValueType.FP32)
+ {
+ vector_len = vector.Length / sizeof(float);
+ }
+ else if (vectorType == VectorValueType.XB8)
+ {
+ vector_len = vector.Length;
+ }
+ else
+ {
+ throw new NotImplementedException($"{vectorType}");
+ }
+
+ var attributes_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(attributes));
+ var attributes_len = attributes.Length;
+
+ return NativeDiskANNMethods.insert(context, index, (nint)id_data, (nuint)id_len, vectorType, (nint)vector_data, (nuint)vector_len, (nint)attributes_data, (nuint)attributes_len) == 1;
+ }
+
+ public bool Remove(ulong context, nint index, ReadOnlySpan id)
+ {
+ var id_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(id));
+ var id_len = id.Length;
+
+ return NativeDiskANNMethods.remove(context, index, (nint)id_data, (nuint)id_len) == 1;
+ }
+
+ public int SearchVector(
+ ulong context,
+ nint index,
+ VectorValueType vectorType,
+ ReadOnlySpan vector,
+ float delta,
+ int searchExplorationFactor,
+ ReadOnlySpan filter,
+ int maxFilteringEffort,
+ Span outputIds,
+ Span outputDistances,
+ out nint continuation
+ )
+ {
+ var vector_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(vector));
+ int vector_len;
+
+ if (vectorType == VectorValueType.FP32)
+ {
+ vector_len = vector.Length / sizeof(float);
+ }
+ else if (vectorType == VectorValueType.XB8)
+ {
+ vector_len = vector.Length;
+ }
+ else
+ {
+ throw new NotImplementedException($"{vectorType}");
+ }
+
+ var filter_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(filter));
+ var filter_len = filter.Length;
+
+ var output_ids = Unsafe.AsPointer(ref MemoryMarshal.GetReference(outputIds));
+ var output_ids_len = outputIds.Length;
+
+ var output_distances = Unsafe.AsPointer(ref MemoryMarshal.GetReference(outputDistances));
+ var output_distances_len = outputDistances.Length;
+
+
+ continuation = 0;
+ ref var continuationRef = ref continuation;
+ var continuationAddr = (nint)Unsafe.AsPointer(ref continuationRef);
+
+ return NativeDiskANNMethods.search_vector(
+ context,
+ index,
+ vectorType,
+ (nint)vector_data,
+ (nuint)vector_len,
+ delta,
+ searchExplorationFactor,
+ (nint)filter_data,
+ (nuint)filter_len,
+ (nuint)maxFilteringEffort,
+ (nint)output_ids,
+ (nuint)output_ids_len,
+ (nint)output_distances,
+ (nuint)output_distances_len,
+ continuationAddr
+ );
+ }
+
+ public int SearchElement(
+ ulong context,
+ nint index,
+ ReadOnlySpan id,
+ float delta,
+ int searchExplorationFactor,
+ ReadOnlySpan filter,
+ int maxFilteringEffort,
+ Span outputIds,
+ Span outputDistances,
+ out nint continuation
+ )
+ {
+ var id_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(id));
+ var id_len = id.Length;
+
+ var filter_data = Unsafe.AsPointer(ref MemoryMarshal.GetReference(filter));
+ var filter_len = filter.Length;
+
+ var output_ids = Unsafe.AsPointer(ref MemoryMarshal.GetReference(outputIds));
+ var output_ids_len = outputIds.Length;
+
+ var output_distances = Unsafe.AsPointer(ref MemoryMarshal.GetReference(outputDistances));
+ var output_distances_len = outputDistances.Length;
+
+ continuation = 0;
+ ref var continuationRef = ref continuation;
+ var continuationAddr = (nint)Unsafe.AsPointer(ref continuationRef);
+
+ return NativeDiskANNMethods.search_element(
+ context,
+ index,
+ (nint)id_data,
+ (nuint)id_len,
+ delta,
+ searchExplorationFactor,
+ (nint)filter_data,
+ (nuint)filter_len,
+ (nuint)maxFilteringEffort,
+ (nint)output_ids,
+ (nuint)output_ids_len,
+ (nint)output_distances,
+ (nuint)output_distances_len,
+ continuationAddr
+ );
+ }
+
+ public int ContinueSearch(ulong context, nint index, nint continuation, Span outputIds, Span outputDistances, out nint newContinuation)
+ {
+ throw new NotImplementedException();
+ }
+
+ public bool TryGetEmbedding(ulong context, nint index, ReadOnlySpan id, Span dimensions)
+ {
+ throw new NotImplementedException();
+ }
+ }
+
+ public static partial class NativeDiskANNMethods
+ {
+ const string DISKANN_GARNET = "diskann_garnet";
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial nint create_index(
+ ulong context,
+ uint dimensions,
+ uint reduceDims,
+ VectorQuantType quantType,
+ uint buildExplorationFactor,
+ uint numLinks,
+ nint readCallback,
+ nint writeCallback,
+ nint deleteCallback,
+ nint readModifyWriteCallback
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial void drop_index(
+ ulong context,
+ nint index
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial byte insert(
+ ulong context,
+ nint index,
+ nint id_data,
+ nuint id_len,
+ VectorValueType vector_value_type,
+ nint vector_data,
+ nuint vector_len,
+ nint attribute_data,
+ nuint attribute_len
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial byte remove(
+ ulong context,
+ nint index,
+ nint id_data,
+ nuint id_len
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial byte set_attribute(
+ ulong context,
+ nint index,
+ nint id_data,
+ nuint id_len,
+ nint attribute_data,
+ nuint attribute_len
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial int search_vector(
+ ulong context,
+ nint index,
+ VectorValueType vector_value_type,
+ nint vector_data,
+ nuint vector_len,
+ float delta,
+ int search_exploration_factor,
+ nint filter_data,
+ nuint filter_len,
+ nuint max_filtering_effort,
+ nint output_ids,
+ nuint output_ids_len,
+ nint output_distances,
+ nuint output_distances_len,
+ nint continuation
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial int search_element(
+ ulong context,
+ nint index,
+ nint id_data,
+ nuint id_len,
+ float delta,
+ int search_exploration_factor,
+ nint filter_data,
+ nuint filter_len,
+ nuint max_filtering_effort,
+ nint output_ids,
+ nuint output_ids_len,
+ nint output_distances,
+ nuint output_distances_len,
+ nint continuation
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial int continue_search(
+ ulong context,
+ nint index,
+ nint continuation,
+ nint output_ids,
+ nuint output_ids_len,
+ nint output_distances,
+ nuint output_distances_len,
+ nint new_continuation
+ );
+
+ [LibraryImport(DISKANN_GARNET)]
+ public static partial ulong card(
+ ulong context,
+ nint index
+ );
+ }
+}
\ No newline at end of file
diff --git a/libs/server/Resp/Vector/RespServerSessionVectors.cs b/libs/server/Resp/Vector/RespServerSessionVectors.cs
new file mode 100644
index 00000000000..dd44f1865f5
--- /dev/null
+++ b/libs/server/Resp/Vector/RespServerSessionVectors.cs
@@ -0,0 +1,1081 @@
+// Copyright (c) Microsoft Corporation.
+// Licensed under the MIT license.
+using System;
+using System.Buffers;
+using System.Buffers.Binary;
+using System.Runtime.InteropServices;
+using Garnet.common;
+using Tsavorite.core;
+
+namespace Garnet.server
+{
+ internal sealed unsafe partial class RespServerSession : ServerSessionBase
+ {
+ private bool NetworkVADD(ref TGarnetApi storageApi)
+ where TGarnetApi : IGarnetApi
+ {
+ // VADD key [REDUCE dim] (FP32 | XB8 | VALUES num) vector element [CAS] [NOQUANT | Q8 | BIN | XPREQ8] [EF build-exploration-factor] [SETATTR attributes] [M numlinks]
+ //
+ // XB8 is a non-Redis extension, stands for: eXtension Binary 8-bit values - encodes [0, 255] per dimension
+ // XPREQ8 is a non-Redis extension, stands for: eXtension PREcalculated Quantization 8-bit - requests no quantization on pre-calculated [0, 255] values
+
+ const int MinM = 4;
+ const int MaxM = 4_096;
+
+ if (!storageSession.vectorManager.IsEnabled)
+ {
+ return AbortWithErrorMessage("ERR Vector Set (preview) commands are not enabled");
+ }
+
+ // key FP32|VALUES vector element
+ if (parseState.Count < 4)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ ref var key = ref parseState.GetArgSliceByRef(0);
+
+ var curIx = 1;
+
+ var reduceDim = 0;
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("REDUCE"u8))
+ {
+ curIx++;
+ if (!parseState.TryGetInt(curIx, out var reduceDimValue) || reduceDimValue <= 0)
+ {
+ return AbortWithErrorMessage("REDUCE dimension must be > 0"u8);
+ }
+
+ reduceDim = reduceDimValue;
+ curIx++;
+ }
+
+ var valueType = VectorValueType.Invalid;
+ byte[] rentedValues = null;
+ Span values = stackalloc byte[64 * sizeof(float)];
+
+ try
+ {
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("FP32"u8))
+ {
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ var asBytes = parseState.GetArgSliceByRef(curIx).Span;
+ if ((asBytes.Length % sizeof(float)) != 0)
+ {
+ return AbortWithErrorMessage("ERR invalid vector specification");
+ }
+
+ curIx++;
+ valueType = VectorValueType.FP32;
+ values = asBytes;
+ }
+ else if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("VALUES"u8))
+ {
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var valueCount) || valueCount <= 0)
+ {
+ return AbortWithErrorMessage("ERR invalid vector specification");
+ }
+ curIx++;
+
+ if (valueCount * sizeof(float) > values.Length)
+ {
+ values = rentedValues = ArrayPool.Shared.Rent(valueCount * sizeof(float));
+ }
+ values = values[..(valueCount * sizeof(float))];
+
+ if (curIx + valueCount > parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ valueType = VectorValueType.FP32;
+ var floatValues = MemoryMarshal.Cast(values);
+
+ for (var valueIx = 0; valueIx < valueCount; valueIx++)
+ {
+ if (!parseState.TryGetFloat(curIx, out floatValues[valueIx]))
+ {
+ return AbortWithErrorMessage("ERR invalid vector specification");
+ }
+
+ curIx++;
+ }
+ }
+ else if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("XB8"u8))
+ {
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ var asBytes = parseState.GetArgSliceByRef(curIx).Span;
+ curIx++;
+
+ valueType = VectorValueType.XB8;
+ values = asBytes;
+ }
+
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VADD");
+ }
+
+ var element = parseState.GetArgSliceByRef(curIx);
+ curIx++;
+
+ // Order for everything after element is unspecified
+ var cas = false;
+ VectorQuantType? quantType = null;
+ int? buildExplorationFactor = null;
+ ArgSlice? attributes = null;
+ int? numLinks = null;
+
+ while (curIx < parseState.Count)
+ {
+ // REDUCE is illegal after values, no matter how specified
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("REDUCE"u8))
+ {
+ return AbortWithErrorMessage("ERR invalid option after element");
+ }
+
+ // Look for CAS
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("CAS"u8))
+ {
+ if (cas)
+ {
+ return AbortWithErrorMessage("CAS specified multiple times");
+ }
+
+ // We ignore CAS, just remember we saw it
+ cas = true;
+ curIx++;
+
+ continue;
+ }
+
+ // Look for quantizer specs
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("NOQUANT"u8))
+ {
+ if (quantType != null)
+ {
+ return AbortWithErrorMessage("Quantization specified multiple times");
+ }
+
+ quantType = VectorQuantType.NoQuant;
+ curIx++;
+
+ continue;
+ }
+ else if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("Q8"u8))
+ {
+ if (quantType != null)
+ {
+ return AbortWithErrorMessage("Quantization specified multiple times");
+ }
+
+ quantType = VectorQuantType.Q8;
+ curIx++;
+
+ continue;
+ }
+ else if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("BIN"u8))
+ {
+ if (quantType != null)
+ {
+ return AbortWithErrorMessage("Quantization specified multiple times");
+ }
+
+ quantType = VectorQuantType.Bin;
+ curIx++;
+
+ continue;
+ }
+ else if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("XPREQ8"u8))
+ {
+ if (quantType != null)
+ {
+ return AbortWithErrorMessage("Quantization specified multiple times");
+ }
+
+ quantType = VectorQuantType.XPreQ8;
+ curIx++;
+
+ continue;
+ }
+
+ // Look for build-exploration-factor
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("EF"u8))
+ {
+ if (buildExplorationFactor != null)
+ {
+ return AbortWithErrorMessage("EF specified multiple times");
+ }
+
+ curIx++;
+
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithErrorMessage("ERR invalid option after element");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var buildExplorationFactorNonNull) || buildExplorationFactorNonNull <= 0)
+ {
+ return AbortWithErrorMessage("ERR invalid EF");
+ }
+
+ buildExplorationFactor = buildExplorationFactorNonNull;
+ curIx++;
+ continue;
+ }
+
+ // Look for attributes
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("SETATTR"u8))
+ {
+ if (attributes != null)
+ {
+ return AbortWithErrorMessage("SETATTR specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithErrorMessage("ERR invalid option after element");
+ }
+
+ attributes = parseState.GetArgSliceByRef(curIx);
+ curIx++;
+
+ // You might think we need to validate attributes, but Redis actually lets anything through
+
+ continue;
+ }
+
+ // Look for num links
+ if (parseState.GetArgSliceByRef(curIx).Span.EqualsUpperCaseSpanIgnoringCase("M"u8))
+ {
+ if (numLinks != null)
+ {
+ return AbortWithErrorMessage("M specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithErrorMessage("ERR invalid option after element");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var numLinksNonNull) || numLinksNonNull < MinM || numLinksNonNull > MaxM)
+ {
+ return AbortWithErrorMessage("ERR invalid M");
+ }
+
+ numLinks = numLinksNonNull;
+ curIx++;
+
+ continue;
+ }
+
+ // Didn't recognize this option, error out
+ return AbortWithErrorMessage("ERR invalid option after element");
+ }
+
+ // Default unspecified options
+ quantType ??= VectorQuantType.Q8;
+ buildExplorationFactor ??= 200;
+ attributes ??= default;
+ numLinks ??= 16;
+
+ // We need to reject these HERE because validation during create_index is very awkward
+ GarnetStatus res;
+ VectorManagerResult result;
+ ReadOnlySpan customErrMsg;
+ if (quantType == VectorQuantType.XPreQ8 && reduceDim != 0)
+ {
+ result = VectorManagerResult.BadParams;
+ res = GarnetStatus.OK;
+ customErrMsg = default;
+ }
+ else
+ {
+ res = storageApi.VectorSetAdd(key, reduceDim, valueType, ArgSlice.FromPinnedSpan(values), element, quantType.Value, buildExplorationFactor.Value, attributes.Value, numLinks.Value, out result, out customErrMsg);
+ }
+
+ if (res == GarnetStatus.OK)
+ {
+ if (result == VectorManagerResult.OK)
+ {
+ if (respProtocolVersion == 3)
+ {
+ while (!RespWriteUtils.TryWriteTrue(ref dcurr, dend))
+ SendAndReset();
+ }
+ else
+ {
+ while (!RespWriteUtils.TryWriteInt32(1, ref dcurr, dend))
+ SendAndReset();
+ }
+ }
+ else if (result == VectorManagerResult.Duplicate)
+ {
+ if (respProtocolVersion == 3)
+ {
+ while (!RespWriteUtils.TryWriteFalse(ref dcurr, dend))
+ SendAndReset();
+ }
+ else
+ {
+ while (!RespWriteUtils.TryWriteInt32(0, ref dcurr, dend))
+ SendAndReset();
+ }
+ }
+ else if (result == VectorManagerResult.BadParams)
+ {
+ if (customErrMsg.IsEmpty)
+ {
+ return AbortWithErrorMessage("ERR asked quantization mismatch with existing vector set"u8);
+ }
+
+ return AbortWithErrorMessage(customErrMsg);
+ }
+ }
+ else
+ {
+ return AbortWithErrorMessage($"Unexpected GarnetStatus: {res}");
+ }
+
+ return true;
+ }
+ finally
+ {
+ if (rentedValues != null)
+ {
+ ArrayPool.Shared.Return(rentedValues);
+ }
+ }
+ }
+
+ private bool NetworkVSIM(ref TGarnetApi storageApi)
+ where TGarnetApi : IGarnetApi
+ {
+ const int DefaultResultSetSize = 64;
+ const int DefaultIdSize = sizeof(ulong);
+ const int DefaultAttributeSize = 32;
+
+ // VSIM key (ELE | FP32 | XB8 | VALUES num) (vector | element) [WITHSCORES] [WITHATTRIBS] [COUNT num] [EPSILON delta] [EF search-exploration - factor] [FILTER expression][FILTER-EF max - filtering - effort] [TRUTH][NOTHREAD]
+ //
+ // XB8 is a non-Redis extension, stands for: eXtension Binary 8-bit values - encodes [0, 255] per dimension
+
+ if (!storageSession.vectorManager.IsEnabled)
+ {
+ return AbortWithErrorMessage("ERR Vector Set (preview) commands are not enabled");
+ }
+
+ if (parseState.Count < 3)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ ref var key = ref parseState.GetArgSliceByRef(0);
+ var kind = parseState.GetArgSliceByRef(1);
+
+ var curIx = 2;
+
+ ArgSlice? element;
+
+ VectorValueType valueType = VectorValueType.Invalid;
+ byte[] rentedValues = null;
+ try
+ {
+ Span values = stackalloc byte[64 * sizeof(float)];
+ if (kind.Span.EqualsUpperCaseSpanIgnoringCase("ELE"u8))
+ {
+ element = parseState.GetArgSliceByRef(curIx);
+ values = default;
+ curIx++;
+ }
+ else
+ {
+ element = default;
+ if (kind.Span.EqualsUpperCaseSpanIgnoringCase("FP32"u8))
+ {
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ var asBytes = parseState.GetArgSliceByRef(curIx).Span;
+ if ((asBytes.Length % sizeof(float)) != 0)
+ {
+ return AbortWithErrorMessage("FP32 values must be multiple of 4-bytes in size");
+ }
+
+ valueType = VectorValueType.FP32;
+ values = asBytes;
+ curIx++;
+ }
+ else if (kind.Span.EqualsUpperCaseSpanIgnoringCase("XB8"u8))
+ {
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ var asBytes = parseState.GetArgSliceByRef(curIx).Span;
+
+ valueType = VectorValueType.XB8;
+ values = asBytes;
+ curIx++;
+ }
+ else if (kind.Span.EqualsUpperCaseSpanIgnoringCase("VALUES"u8))
+ {
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var valueCount) || valueCount <= 0)
+ {
+ return AbortWithErrorMessage("VALUES count must > 0");
+ }
+ curIx++;
+
+ if (valueCount * sizeof(float) > values.Length)
+ {
+ values = rentedValues = ArrayPool.Shared.Rent(valueCount * sizeof(float));
+ }
+ values = values[..(valueCount * sizeof(float))];
+
+ if (curIx + valueCount > parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ valueType = VectorValueType.FP32;
+ var floatValues = MemoryMarshal.Cast(values);
+
+ for (var valueIx = 0; valueIx < valueCount; valueIx++)
+ {
+ if (!parseState.TryGetFloat(curIx, out floatValues[valueIx]))
+ {
+ return AbortWithErrorMessage("VALUES value must be valid float");
+ }
+
+ curIx++;
+ }
+ }
+ else
+ {
+ return AbortWithErrorMessage("VSIM expected ELE, FP32, or VALUES");
+ }
+ }
+
+ bool? withScores = null;
+ bool? withAttributes = null;
+ int? count = null;
+ float? delta = null;
+ int? searchExplorationFactor = null;
+ ArgSlice? filter = null;
+ int? maxFilteringEffort = null;
+ var truth = false;
+ var noThread = false;
+
+ while (curIx < parseState.Count)
+ {
+ // Check for withScores
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("WITHSCORES"u8))
+ {
+ if (withScores != null)
+ {
+ return AbortWithErrorMessage("WITHSCORES specified multiple times");
+ }
+
+ withScores = true;
+ curIx++;
+ continue;
+ }
+
+ // Check for withAttributes
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("WITHATTRIBS"u8))
+ {
+ if (withAttributes != null)
+ {
+ return AbortWithErrorMessage("WITHATTRIBS specified multiple times");
+ }
+
+ withAttributes = true;
+ curIx++;
+ continue;
+ }
+
+ // Check for count
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("COUNT"u8))
+ {
+ if (count != null)
+ {
+ return AbortWithErrorMessage("COUNT specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var countNonNull) || countNonNull < 0)
+ {
+ return AbortWithErrorMessage("COUNT must be integer >= 0");
+ }
+
+ count = countNonNull;
+ curIx++;
+ continue;
+ }
+
+ // Check for delta
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("EPSILON"u8))
+ {
+ if (delta != null)
+ {
+ return AbortWithErrorMessage("EPSILON specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ if (!parseState.TryGetFloat(curIx, out var deltaNonNull) || deltaNonNull <= 0)
+ {
+ return AbortWithErrorMessage("EPSILON must be float > 0");
+ }
+
+ delta = deltaNonNull;
+ curIx++;
+ continue;
+ }
+
+ // Check for search exploration factor
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("EF"u8))
+ {
+ if (searchExplorationFactor != null)
+ {
+ return AbortWithErrorMessage("EF specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var searchExplorationFactorNonNull) || searchExplorationFactorNonNull < 0)
+ {
+ return AbortWithErrorMessage("EF must be >= 0");
+ }
+
+ searchExplorationFactor = searchExplorationFactorNonNull;
+ curIx++;
+ continue;
+ }
+
+ // Check for filter
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("FILTER"u8))
+ {
+ if (filter != null)
+ {
+ return AbortWithErrorMessage("FILTER specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ filter = parseState.GetArgSliceByRef(curIx);
+ curIx++;
+
+ // TODO: validate filter
+
+ continue;
+ }
+
+ // Check for max filtering effort
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("FILTER-EF"u8))
+ {
+ if (maxFilteringEffort != null)
+ {
+ return AbortWithErrorMessage("FILTER-EF specified multiple times");
+ }
+
+ curIx++;
+ if (curIx >= parseState.Count)
+ {
+ return AbortWithWrongNumberOfArguments("VSIM");
+ }
+
+ if (!parseState.TryGetInt(curIx, out var maxFilteringEffortNonNull) || maxFilteringEffortNonNull < 0)
+ {
+ return AbortWithErrorMessage("FILTER-EF must be >= 0");
+ }
+
+ maxFilteringEffort = maxFilteringEffortNonNull;
+ curIx++;
+ continue;
+ }
+
+ // Check for truth
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("TRUTH"u8))
+ {
+ if (truth)
+ {
+
+ }
+
+ // TODO: should we implement TRUTH?
+ truth = true;
+ curIx++;
+ continue;
+ }
+
+ // Check for no thread
+ if (parseState.GetArgSliceByRef(curIx).ReadOnlySpan.EqualsUpperCaseSpanIgnoringCase("NOTHREAD"u8))
+ {
+ if (noThread)
+ {
+ return AbortWithErrorMessage("NOTHREAD specified multiple times");
+ }
+
+ // We ignore NOTHREAD
+ noThread = true;
+ curIx++;
+ continue;
+ }
+
+ // Didn't recognize this option, error out
+ return AbortWithErrorMessage("Unknown option");
+ }
+
+ // Default unspecified options
+ withScores ??= false;
+ withAttributes ??= false;
+ count ??= 10;
+ delta ??= 2f;
+ searchExplorationFactor ??= 100;
+ filter ??= default;
+ maxFilteringEffort ??= count.Value * 100;
+
+ // TODO: these stackallocs are dangerous, need logic to avoid stack overflow
+ Span