diff --git a/Version.props b/Version.props
index 20e5abf6635..45fcee605a6 100644
--- a/Version.props
+++ b/Version.props
@@ -1,6 +1,6 @@
- 1.0.83
+ 1.0.84
diff --git a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs
index de849013dab..835e966ba6c 100644
--- a/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs
+++ b/benchmark/BDN.benchmark/Custom/CustomTxnSet.cs
@@ -52,10 +52,10 @@ public override bool Prepare(TGarnetReadApi api, ref CustomProce
valueC = GetNextArg(ref procInput, ref offset);
valueD = GetNextArg(ref procInput, ref offset);
- AddKey(setA, LockType.Exclusive, isObject: false);
- AddKey(setB, LockType.Exclusive, isObject: false);
- AddKey(setC, LockType.Exclusive, isObject: false);
- AddKey(setD, LockType.Exclusive, isObject: false);
+ AddKey(setA, LockType.Exclusive, StoreType.Main);
+ AddKey(setB, LockType.Exclusive, StoreType.Main);
+ AddKey(setC, LockType.Exclusive, StoreType.Main);
+ AddKey(setD, LockType.Exclusive, StoreType.Main);
return true;
}
diff --git a/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs b/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs
deleted file mode 100644
index 0b64944783c..00000000000
--- a/libs/client/ClientSession/GarnetClientSessionClusterExtensions.cs
+++ /dev/null
@@ -1,69 +0,0 @@
-// Copyright (c) Microsoft Corporation.
-// Licensed under the MIT license.
-
-using System;
-using System.Threading;
-using System.Threading.Tasks;
-using Garnet.common;
-using Garnet.networking;
-
-namespace Garnet.client
-{
- ///
- /// Mono-threaded remote client session for Garnet (a session makes a single network connection, and
- /// expects mono-threaded client access, i.e., no concurrent invocations of API by client)
- ///
- public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageConsumer
- {
- static ReadOnlySpan GOSSIP => "GOSSIP"u8;
-
- ///
- /// Send gossip message to corresponding node
- ///
- ///
- ///
- public Task ExecuteGossip(Memory byteArray)
- {
- var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
- tcsQueue.Enqueue(tcs);
- byte* curr = offset;
- byte* next = offset;
- int arraySize = 3;
-
- while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end))
- {
- Flush();
- curr = offset;
- }
- offset = curr;
-
- //1
- while (!RespWriteUtils.TryWriteDirect(CLUSTER, ref curr, end))
- {
- Flush();
- curr = offset;
- }
- offset = curr;
-
- //2
- while (!RespWriteUtils.TryWriteBulkString(GOSSIP, ref curr, end))
- {
- Flush();
- curr = offset;
- }
- offset = curr;
-
- //3
- while (!RespWriteUtils.TryWriteBulkString(byteArray.Span, ref curr, end))
- {
- Flush();
- curr = offset;
- }
- offset = curr;
-
- Flush();
- Interlocked.Increment(ref numCommands);
- return tcs.Task;
- }
- }
-}
\ No newline at end of file
diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs
index 2f9f060c415..3beac1a77ff 100644
--- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs
+++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs
@@ -31,6 +31,7 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo
///
///
///
+ ///
public Task ExecuteReplicaSync(string nodeId, string primary_replid, byte[] checkpointEntryData, long aofBeginAddress, long aofTailAddress)
{
var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
@@ -112,6 +113,7 @@ public Task ExecuteReplicaSync(string nodeId, string primary_replid, byt
///
///
///
+ ///
public Task ExecuteSendCkptMetadata(Memory fileTokenBytes, int fileType, Memory data)
{
var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
@@ -178,6 +180,7 @@ public Task ExecuteSendCkptMetadata(Memory fileTokenBytes, int fil
///
///
///
+ ///
public Task ExecuteSendFileSegments(Memory fileTokenBytes, int fileType, long startAddress, Span data, int segmentId = -1)
{
var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
@@ -257,19 +260,19 @@ public Task ExecuteSendFileSegments(Memory fileTokenBytes, int fil
/// Signal replica to recover
///
///
- ///
///
///
///
///
///
///
- public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool sendObjectStoreCheckpoint, bool replayAOF, string primary_replid, byte[] checkpointEntryData, long beginAddress, long tailAddress)
+ ///
+ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool replayAOF, string primary_replid, byte[] checkpointEntryData, long beginAddress, long tailAddress)
{
var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
tcsQueue.Enqueue(tcs);
byte* curr = offset;
- int arraySize = 9;
+ int arraySize = 8;
while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end))
{
@@ -303,14 +306,6 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
offset = curr;
//4
- while (!RespWriteUtils.TryWriteBulkString(sendObjectStoreCheckpoint ? "1"u8 : "0"u8, ref curr, end))
- {
- Flush();
- curr = offset;
- }
- offset = curr;
-
- //5
while (!RespWriteUtils.TryWriteBulkString(replayAOF ? "1"u8 : "0"u8, ref curr, end))
{
Flush();
@@ -318,7 +313,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
}
offset = curr;
- //6
+ //5
while (!RespWriteUtils.TryWriteAsciiBulkString(primary_replid, ref curr, end))
{
Flush();
@@ -326,7 +321,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
}
offset = curr;
- //7
+ //6
while (!RespWriteUtils.TryWriteBulkString(checkpointEntryData, ref curr, end))
{
Flush();
@@ -334,7 +329,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
}
offset = curr;
- //8
+ //7
while (!RespWriteUtils.TryWriteArrayItem(beginAddress, ref curr, end))
{
Flush();
@@ -342,7 +337,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
}
offset = curr;
- //9
+ //8
while (!RespWriteUtils.TryWriteArrayItem(tailAddress, ref curr, end))
{
Flush();
@@ -360,6 +355,7 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
///
///
///
+ ///
public Task ExecuteAttachSync(byte[] syncMetadata)
{
var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously);
@@ -408,6 +404,7 @@ public Task ExecuteAttachSync(byte[] syncMetadata)
///
///
///
+ ///
public void SetClusterSyncHeader(string sourceNodeId, bool isMainStore)
{
// Unlike Migration, where we don't know at the time of header initialization if we have a record or not, in Replication
diff --git a/libs/cluster/Server/ClusterManagerSlotState.cs b/libs/cluster/Server/ClusterManagerSlotState.cs
index 5041f656d9a..3f35e1c029f 100644
--- a/libs/cluster/Server/ClusterManagerSlotState.cs
+++ b/libs/cluster/Server/ClusterManagerSlotState.cs
@@ -13,10 +13,13 @@
namespace Garnet.cluster
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
ObjectAllocator>>>;
///
@@ -482,7 +485,7 @@ public static unsafe void DeleteKeysInSlotsFromMainStore(BasicGarnetApi BasicGar
var key = iter.Key;
var s = HashSlotUtils.HashSlot(key);
if (slots.Contains(s))
- _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Main);
+ _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key));
}
}
@@ -499,7 +502,7 @@ public static unsafe void DeleteKeysInSlotsFromObjectStore(BasicGarnetApi BasicG
var key = iterObject.Key;
var s = HashSlotUtils.HashSlot(key);
if (slots.Contains(s))
- _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key), StoreType.Object);
+ _ = BasicGarnetApi.DELETE(PinnedSpanByte.FromPinnedSpan(key));
}
}
}
diff --git a/libs/cluster/Server/ClusterManagerWorkerState.cs b/libs/cluster/Server/ClusterManagerWorkerState.cs
index 8d83c355ca5..15f4f308a94 100644
--- a/libs/cluster/Server/ClusterManagerWorkerState.cs
+++ b/libs/cluster/Server/ClusterManagerWorkerState.cs
@@ -100,6 +100,10 @@ public ReadOnlySpan TryReset(bool soft, int expirySeconds = 60)
try
{
SuspendConfigMerge();
+
+ // Reset recovery operations before proceeding with reset
+ clusterProvider.replicationManager.ResetRecovery();
+
var resp = CmdStrings.RESP_OK;
while (true)
{
@@ -113,8 +117,9 @@ public ReadOnlySpan TryReset(bool soft, int expirySeconds = 60)
this.clusterConnectionStore.CloseAll();
var newNodeId = soft ? current.LocalNodeId : Generator.CreateHexId();
- var address = current.LocalNodeIp;
- var port = current.LocalNodePort;
+ var endpoint = clusterProvider.storeWrapper.GetClusterEndpoint();
+ var address = endpoint.Address.ToString();
+ var port = endpoint.Port;
var configEpoch = soft ? current.LocalNodeConfigEpoch : 0;
var expiry = DateTimeOffset.UtcNow.Ticks + TimeSpan.FromSeconds(expirySeconds).Ticks;
diff --git a/libs/cluster/Server/ClusterProvider.cs b/libs/cluster/Server/ClusterProvider.cs
index 3c060d35273..90bd30183b5 100644
--- a/libs/cluster/Server/ClusterProvider.cs
+++ b/libs/cluster/Server/ClusterProvider.cs
@@ -16,10 +16,13 @@
namespace Garnet.cluster
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
ObjectAllocator>>>;
///
@@ -157,7 +160,6 @@ public void FlushConfig()
public void FlushDB(bool unsafeTruncateLog = false)
{
storeWrapper.store.Log.ShiftBeginAddress(storeWrapper.store.Log.TailAddress, truncateLog: unsafeTruncateLog);
- storeWrapper.objectStore?.Log.ShiftBeginAddress(storeWrapper.objectStore.Log.TailAddress, truncateLog: unsafeTruncateLog);
}
///
@@ -171,12 +173,6 @@ public void SafeTruncateAOF(bool full, long CheckpointCoveredAofAddress, Guid st
entry.metadata.storeCheckpointCoveredAofAddress = CheckpointCoveredAofAddress;
entry.metadata.storePrimaryReplId = replicationManager.PrimaryReplId;
- entry.metadata.objectStoreVersion = serverOptions.DisableObjects ? -1 : storeWrapper.objectStore.CurrentVersion;
- entry.metadata.objectStoreHlogToken = serverOptions.DisableObjects ? default : objectStoreCheckpointToken;
- entry.metadata.objectStoreIndexToken = serverOptions.DisableObjects ? default : objectStoreCheckpointToken;
- entry.metadata.objectCheckpointCoveredAofAddress = CheckpointCoveredAofAddress;
- entry.metadata.objectStorePrimaryReplId = replicationManager.PrimaryReplId;
-
// Keep track of checkpoints for replica
// Used to delete old checkpoints and cleanup and also cleanup during attachment to new primary
replicationManager.AddCheckpointEntry(entry, full);
@@ -241,8 +237,6 @@ public MetricsItem[] GetReplicationInfo()
new("second_repl_offset", replication_offset2),
new("store_current_safe_aof_address", clusterEnabled ? replicationManager.StoreCurrentSafeAofAddress.ToString() : "N/A"),
new("store_recovered_safe_aof_address", clusterEnabled ? replicationManager.StoreRecoveredSafeAofTailAddress.ToString() : "N/A"),
- new("object_store_current_safe_aof_address", clusterEnabled && !serverOptions.DisableObjects ? replicationManager.ObjectStoreCurrentSafeAofAddress.ToString() : "N/A"),
- new("object_store_recovered_safe_aof_address", clusterEnabled && !serverOptions.DisableObjects ? replicationManager.ObjectStoreRecoveredSafeAofTailAddress.ToString() : "N/A"),
new("recover_status", replicationManager.currentRecoveryStatus.ToString()),
new("last_failover_state", !clusterEnabled ? FailoverUtils.GetFailoverStatus(FailoverStatus.NO_FAILOVER) : failoverManager.GetLastFailoverStatus())
};
@@ -427,15 +421,10 @@ public void ExtractKeySpecs(RespCommandsInfo commandInfo, RespCommand cmd, ref S
public void ClusterPublish(RespCommand cmd, ref Span channel, ref Span message)
=> clusterManager.TryClusterPublish(cmd, ref channel, ref message);
- internal GarnetClusterCheckpointManager GetReplicationLogCheckpointManager(StoreType storeType)
+ internal GarnetClusterCheckpointManager GetReplicationLogCheckpointManager()
{
Debug.Assert(serverOptions.EnableCluster);
- return storeType switch
- {
- StoreType.Main => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager,
- StoreType.Object => (GarnetClusterCheckpointManager)storeWrapper.objectStore?.CheckpointManager,
- _ => throw new Exception($"GetCkptManager: unexpected state {storeType}")
- };
+ return (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager;
}
///
diff --git a/libs/cluster/Server/Migration/MigrateSessionKeys.cs b/libs/cluster/Server/Migration/MigrateSessionKeys.cs
index 687f5dd6367..629f95c8f95 100644
--- a/libs/cluster/Server/Migration/MigrateSessionKeys.cs
+++ b/libs/cluster/Server/Migration/MigrateSessionKeys.cs
@@ -104,11 +104,8 @@ public bool MigrateKeys()
return false;
// Migrate object store keys
- if (!clusterProvider.serverOptions.DisableObjects)
- {
- if (!MigrateKeysFromObjectStore())
- return false;
- }
+ if (!MigrateKeysFromObjectStore())
+ return false;
}
catch (Exception ex)
{
diff --git a/libs/cluster/Server/Migration/MigrateSessionSlots.cs b/libs/cluster/Server/Migration/MigrateSessionSlots.cs
index cc139281a99..5406c79141d 100644
--- a/libs/cluster/Server/Migration/MigrateSessionSlots.cs
+++ b/libs/cluster/Server/Migration/MigrateSessionSlots.cs
@@ -33,17 +33,6 @@ public async Task MigrateSlotsDriverInline()
var success = await CreateAndRunMigrateTasks(StoreType.Main, storeBeginAddress, storeTailAddress, mainStorePageSize);
if (!success) return false;
- // Send object store
- if (!clusterProvider.serverOptions.DisableObjects)
- {
- var objectStoreBeginAddress = clusterProvider.storeWrapper.objectStore.Log.BeginAddress;
- var objectStoreTailAddress = clusterProvider.storeWrapper.objectStore.Log.TailAddress;
- var objectStorePageSize = 1 << clusterProvider.serverOptions.ObjectStorePageSizeBits();
- logger?.LogWarning("Object Store migrate scan range [{objectStoreBeginAddress}, {objectStoreTailAddress}]", objectStoreBeginAddress, objectStoreTailAddress);
- success = await CreateAndRunMigrateTasks(StoreType.Object, objectStoreBeginAddress, objectStoreTailAddress, objectStorePageSize);
- if (!success) return false;
- }
-
return true;
async Task CreateAndRunMigrateTasks(StoreType storeType, long beginAddress, long tailAddress, int pageSize)
diff --git a/libs/cluster/Server/Replication/CheckpointEntry.cs b/libs/cluster/Server/Replication/CheckpointEntry.cs
index a6ba395f633..9c57eb7c2af 100644
--- a/libs/cluster/Server/Replication/CheckpointEntry.cs
+++ b/libs/cluster/Server/Replication/CheckpointEntry.cs
@@ -22,22 +22,13 @@ public static void LogCheckpointEntry(this ILogger logger, LogLevel logLevel, st
"storeHlogToken: {storeHlogToken}\n" +
"storeIndexToken: {storeIndexToken}\n" +
"storeCheckpointCoveredAofAddress: {storeCheckpointCoveredAofAddress}\n" +
- "------------------------------------------------------------------------\n" +
- "objectStoreVersion:{objectStoreVersion}\n" +
- "objectStoreHlogToken:{objectStoreHlogToken}\n" +
- "objectStoreIndexToken:{objectStoreIndexToken}\n" +
- "objectCheckpointCoveredAofAddress:{objectCheckpointCoveredAofAddress}\n" +
"------------------------------------------------------------------------\n",
msg,
entry._lock,
entry.metadata.storeVersion,
entry.metadata.storeHlogToken,
entry.metadata.storeIndexToken,
- entry.metadata.storeCheckpointCoveredAofAddress,
- entry.metadata.objectStoreVersion,
- entry.metadata.objectStoreHlogToken,
- entry.metadata.objectStoreIndexToken,
- entry.metadata.objectCheckpointCoveredAofAddress);
+ entry.metadata.storeCheckpointCoveredAofAddress);
}
}
@@ -55,7 +46,7 @@ public CheckpointEntry()
}
public long GetMinAofCoveredAddress()
- => Math.Max(Math.Min(metadata.storeCheckpointCoveredAofAddress, metadata.objectCheckpointCoveredAofAddress), LogAddress.FirstValidAddress);
+ => Math.Max(metadata.storeCheckpointCoveredAofAddress, LogAddress.FirstValidAddress);
///
/// Indicate addition of new reader by trying to increment reader counter
@@ -90,8 +81,6 @@ public bool ContainsSharedToken(CheckpointEntry entry, CheckpointFileType fileTy
{
CheckpointFileType.STORE_HLOG => metadata.storeHlogToken.Equals(entry.metadata.storeHlogToken),
CheckpointFileType.STORE_INDEX => metadata.storeIndexToken.Equals(entry.metadata.storeIndexToken),
- CheckpointFileType.OBJ_STORE_HLOG => metadata.objectStoreHlogToken.Equals(entry.metadata.objectStoreHlogToken),
- CheckpointFileType.OBJ_STORE_INDEX => metadata.objectStoreIndexToken.Equals(entry.metadata.objectStoreIndexToken),
_ => throw new Exception($"Option {fileType} not supported")
};
}
@@ -118,18 +107,6 @@ public byte[] ToByteArray()
writer.Write(metadata.storePrimaryReplId == null ? 0 : 1);
if (metadata.storePrimaryReplId != null) writer.Write(metadata.storePrimaryReplId);
- // Write checkpoint entry data for object store
- writer.Write(metadata.objectStoreVersion);
- byteBuffer = metadata.objectStoreHlogToken.ToByteArray();
- writer.Write(byteBuffer.Length);
- writer.Write(byteBuffer);
- byteBuffer = metadata.objectStoreIndexToken.ToByteArray();
- writer.Write(byteBuffer.Length);
- writer.Write(byteBuffer);
- writer.Write(metadata.objectCheckpointCoveredAofAddress);
- writer.Write(metadata.objectStorePrimaryReplId == null ? 0 : 1);
- if (metadata.objectStorePrimaryReplId != null) writer.Write(metadata.objectStorePrimaryReplId);
-
var byteArray = ms.ToArray();
writer.Dispose();
ms.Dispose();
@@ -154,13 +131,7 @@ public static CheckpointEntry FromByteArray(byte[] serialized)
storeHlogToken = new Guid(reader.ReadBytes(reader.ReadInt32())),
storeIndexToken = new Guid(reader.ReadBytes(reader.ReadInt32())),
storeCheckpointCoveredAofAddress = reader.ReadInt64(),
- storePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default,
-
- objectStoreVersion = reader.ReadInt64(),
- objectStoreHlogToken = new Guid(reader.ReadBytes(reader.ReadInt32())),
- objectStoreIndexToken = new Guid(reader.ReadBytes(reader.ReadInt32())),
- objectCheckpointCoveredAofAddress = reader.ReadInt64(),
- objectStorePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default
+ storePrimaryReplId = reader.ReadInt32() > 0 ? reader.ReadString() : default
}
};
diff --git a/libs/cluster/Server/Replication/CheckpointFileType.cs b/libs/cluster/Server/Replication/CheckpointFileType.cs
index da2ac9690ba..e20bb1e28fa 100644
--- a/libs/cluster/Server/Replication/CheckpointFileType.cs
+++ b/libs/cluster/Server/Replication/CheckpointFileType.cs
@@ -1,9 +1,6 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT license.
-using System;
-using Garnet.server;
-
namespace Garnet.cluster
{
///
@@ -31,49 +28,5 @@ enum CheckpointFileType : byte
/// Store Snapshot
///
STORE_SNAPSHOT,
- ///
- /// Object Store Hybrid Log - Main
- ///
- OBJ_STORE_HLOG,
- ///
- /// Object Store Hybrid Log - Object
- ///
- OBJ_STORE_HLOG_OBJ,
- ///
- /// Object Store Delta Log
- ///
- OBJ_STORE_DLOG,
- ///
- /// Object Store Index
- ///
- OBJ_STORE_INDEX,
- ///
- /// Object Store Snapshot - Main
- ///
- OBJ_STORE_SNAPSHOT,
- ///
- /// Object Store Snapshot - Object
- ///
- OBJ_STORE_SNAPSHOT_OBJ,
- }
-
- static class CheckpointFileTypeExtensions
- {
- public static StoreType ToStoreType(this CheckpointFileType type)
- {
- return type switch
- {
- CheckpointFileType.STORE_HLOG or
- CheckpointFileType.STORE_DLOG or
- CheckpointFileType.STORE_INDEX or
- CheckpointFileType.STORE_SNAPSHOT => StoreType.Main,
- CheckpointFileType.OBJ_STORE_HLOG or
- CheckpointFileType.OBJ_STORE_DLOG or
- CheckpointFileType.OBJ_STORE_INDEX or
- CheckpointFileType.OBJ_STORE_SNAPSHOT or
- CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ => StoreType.Object,
- _ => throw new Exception($"ToStoreType: unexpected state {type}")
- };
- }
}
}
\ No newline at end of file
diff --git a/libs/cluster/Server/Replication/CheckpointStore.cs b/libs/cluster/Server/Replication/CheckpointStore.cs
index e3d8cc44045..8616f5e5be5 100644
--- a/libs/cluster/Server/Replication/CheckpointStore.cs
+++ b/libs/cluster/Server/Replication/CheckpointStore.cs
@@ -40,7 +40,7 @@ public void Initialize()
{
head = tail = GetLatestCheckpointEntryFromDisk();
- if (tail.metadata.storeVersion == -1 && tail.metadata.objectStoreVersion == -1)
+ if (tail.metadata.storeVersion == -1)
{
head = tail = null;
}
@@ -48,11 +48,6 @@ public void Initialize()
{
clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredSafeAofAddress = tail.metadata.storeCheckpointCoveredAofAddress;
clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredHistoryId = tail.metadata.storePrimaryReplId;
- if (!storeWrapper.serverOptions.DisableObjects)
- {
- clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredSafeAofAddress = tail.metadata.storeCheckpointCoveredAofAddress;
- clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredHistoryId = tail.metadata.storePrimaryReplId;
- }
}
// This purge does not check for active readers
@@ -85,13 +80,11 @@ public void PurgeAllCheckpointsExceptEntry(CheckpointEntry entry = null)
entry ??= GetLatestCheckpointEntryFromDisk();
if (entry == null) return;
logger?.LogCheckpointEntry(LogLevel.Trace, nameof(PurgeAllCheckpointsExceptEntry), entry);
- PurgeAllCheckpointsExceptTokens(StoreType.Main, entry.metadata.storeHlogToken, entry.metadata.storeIndexToken);
- if (!clusterProvider.serverOptions.DisableObjects)
- PurgeAllCheckpointsExceptTokens(StoreType.Object, entry.metadata.objectStoreHlogToken, entry.metadata.objectStoreIndexToken);
+ PurgeAllCheckpointsExceptTokens(entry.metadata.storeHlogToken, entry.metadata.storeIndexToken);
- void PurgeAllCheckpointsExceptTokens(StoreType storeType, Guid logToken, Guid indexToken)
+ void PurgeAllCheckpointsExceptTokens(Guid logToken, Guid indexToken)
{
- var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(storeType);
+ var ckptManager = clusterProvider.GetReplicationLogCheckpointManager();
// Delete log checkpoints
foreach (var toDeletelogToken in ckptManager.GetLogCheckpointTokens())
@@ -129,7 +122,6 @@ public void AddCheckpointEntry(CheckpointEntry entry, bool fullCheckpoint = fals
{
var lastEntry = tail ?? throw new GarnetException($"Checkpoint history unavailable, need full checkpoint for {entry}");
entry.metadata.storeIndexToken = lastEntry.metadata.storeIndexToken;
- entry.metadata.objectStoreIndexToken = lastEntry.metadata.objectStoreIndexToken;
}
_ = ValidateCheckpointEntry(entry);
@@ -156,9 +148,6 @@ bool ValidateCheckpointEntry(CheckpointEntry entry)
if (!clusterProvider.replicationManager.TryAcquireSettledMetadataForMainStore(entry, out _, out _))
throw new GarnetException("Failed to validate main store metadata at insertion");
- if (!clusterProvider.serverOptions.DisableObjects && !clusterProvider.replicationManager.TryAcquireSettledMetadataForObjectStore(entry, out _, out _))
- throw new GarnetException("Failed to validate object store metadata at insertion");
-
return true;
}
catch (Exception ex)
@@ -191,22 +180,11 @@ private void DeleteOutdatedCheckpoints()
// Below check each checkpoint token separately if it is eligible for deletion
if (!CanDeleteToken(curr, CheckpointFileType.STORE_HLOG))
break;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).DeleteLogCheckpoint(curr.metadata.storeHlogToken);
+ clusterProvider.GetReplicationLogCheckpointManager().DeleteLogCheckpoint(curr.metadata.storeHlogToken);
if (!CanDeleteToken(curr, CheckpointFileType.STORE_INDEX))
break;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).DeleteIndexCheckpoint(curr.metadata.storeIndexToken);
-
- if (!clusterProvider.serverOptions.DisableObjects)
- {
- if (!CanDeleteToken(curr, CheckpointFileType.OBJ_STORE_HLOG))
- break;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).DeleteLogCheckpoint(curr.metadata.objectStoreHlogToken);
-
- if (!CanDeleteToken(curr, CheckpointFileType.OBJ_STORE_INDEX))
- break;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).DeleteIndexCheckpoint(curr.metadata.objectStoreIndexToken);
- }
+ clusterProvider.GetReplicationLogCheckpointManager().DeleteIndexCheckpoint(curr.metadata.storeIndexToken);
logger?.LogCheckpointEntry(LogLevel.Warning, "Deleting outdated checkpoint", curr);
@@ -262,7 +240,6 @@ public bool TryGetLatestCheckpointEntryFromMemory(out CheckpointEntry cEntry)
metadata = new()
{
storeCheckpointCoveredAofAddress = 0,
- objectCheckpointCoveredAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : 0
}
};
_ = cEntry.TryAddReader();
@@ -282,13 +259,8 @@ public bool TryGetLatestCheckpointEntryFromMemory(out CheckpointEntry cEntry)
///
public CheckpointEntry GetLatestCheckpointEntryFromDisk()
{
- Guid objectStoreHLogToken = default;
- Guid objectStoreIndexToken = default;
- var objectStoreVersion = -1L;
storeWrapper.store.GetLatestCheckpointTokens(out var storeHLogToken, out var storeIndexToken, out var storeVersion);
- storeWrapper.objectStore?.GetLatestCheckpointTokens(out objectStoreHLogToken, out objectStoreIndexToken, out objectStoreVersion);
- var (storeCheckpointCoveredAofAddress, storePrimaryReplId) = GetCheckpointCookieMetadata(StoreType.Main, storeHLogToken);
- var (objectCheckpointCoveredAofAddress, objectStorePrimaryReplId) = objectStoreHLogToken == default ? (long.MaxValue, null) : GetCheckpointCookieMetadata(StoreType.Object, objectStoreHLogToken);
+ var (storeCheckpointCoveredAofAddress, storePrimaryReplId) = GetCheckpointCookieMetadata(storeHLogToken);
CheckpointEntry entry = new()
{
@@ -299,21 +271,15 @@ public CheckpointEntry GetLatestCheckpointEntryFromDisk()
storeIndexToken = storeIndexToken,
storeCheckpointCoveredAofAddress = storeCheckpointCoveredAofAddress,
storePrimaryReplId = storePrimaryReplId,
-
- objectStoreVersion = objectStoreVersion,
- objectStoreHlogToken = objectStoreHLogToken,
- objectStoreIndexToken = objectStoreIndexToken,
- objectCheckpointCoveredAofAddress = objectCheckpointCoveredAofAddress,
- objectStorePrimaryReplId = objectStorePrimaryReplId,
}
};
return entry;
- (long RecoveredSafeAofAddress, string RecoveredReplicationId) GetCheckpointCookieMetadata(StoreType storeType, Guid fileToken)
+ (long RecoveredSafeAofAddress, string RecoveredReplicationId) GetCheckpointCookieMetadata(Guid fileToken)
{
if (fileToken == default) return (0, null);
- var ckptManager = clusterProvider.GetReplicationLogCheckpointManager(storeType);
- var pageSizeBits = storeType == StoreType.Main ? clusterProvider.serverOptions.PageSizeBits() : clusterProvider.serverOptions.ObjectStorePageSizeBits();
+ var ckptManager = clusterProvider.GetReplicationLogCheckpointManager();
+ var pageSizeBits = clusterProvider.serverOptions.PageSizeBits();
using (var deltaFileDevice = ckptManager.GetDeltaLogDevice(fileToken))
{
if (deltaFileDevice is not null)
diff --git a/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs b/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs
index 3bf7ec5b5fe..36829e9abb1 100644
--- a/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs
+++ b/libs/cluster/Server/Replication/GarnetClusterCheckpointManager.cs
@@ -58,10 +58,6 @@ public IDevice GetDevice(CheckpointFileType retStateType, Guid fileToken)
CheckpointFileType.STORE_DLOG => GetDeltaLogDevice(fileToken),
CheckpointFileType.STORE_INDEX => GetIndexDevice(fileToken),
CheckpointFileType.STORE_SNAPSHOT => GetSnapshotLogDevice(fileToken),
- CheckpointFileType.OBJ_STORE_DLOG => GetDeltaLogDevice(fileToken),
- CheckpointFileType.OBJ_STORE_INDEX => GetIndexDevice(fileToken),
- CheckpointFileType.OBJ_STORE_SNAPSHOT => GetSnapshotLogDevice(fileToken),
- CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ => GetSnapshotObjectLogDevice(fileToken),
_ => throw new Exception($"RetrieveCheckpointFile: unexpected state{retStateType}")
};
return device;
diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs
index c31f1b4b2bd..69df96fffca 100644
--- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs
+++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs
@@ -30,8 +30,6 @@ internal sealed partial class ReplicaSyncSession
public long currentStoreVersion;
- public long currentObjectStoreVersion;
-
///
/// Pessimistic checkpoint covered AOF address
///
@@ -192,7 +190,6 @@ public bool NeedToFullSync()
var localPrimaryReplId = clusterProvider.replicationManager.PrimaryReplId;
var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal);
var sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion;
- var sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion;
var aofBeginAddress = clusterProvider.storeWrapper.appendOnlyFile.BeginAddress;
var aofTailAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress;
@@ -202,11 +199,10 @@ public bool NeedToFullSync()
// We need to stream checkpoint if any of the following conditions are met:
// 1. Replica has different history than primary
- // 2. Replica has different main store version than primary
- // 3. Replica has different object store version than primary
- // 4. Replica has truncated AOF
- // 5. The AOF to be replayed in case of a partial sync is larger than the specified threshold
- fullSync = sendMainStore || sendObjectStore || outOfRangeAof || aofTooLarge;
+ // 2. Replica has different store version than primary
+ // 3. Replica has truncated AOF
+ // 4. The AOF to be replayed in case of a partial sync is larger than the specified threshold
+ fullSync = sendMainStore || outOfRangeAof || aofTooLarge;
return fullSync;
}
@@ -227,7 +223,6 @@ public async Task BeginAofSync()
originNodeId: clusterProvider.clusterManager.CurrentConfig.LocalNodeId,
currentPrimaryReplId: clusterProvider.replicationManager.PrimaryReplId,
currentStoreVersion: currentStoreVersion,
- currentObjectStoreVersion: currentObjectStoreVersion,
currentAofBeginAddress: currentAofBeginAddress,
currentAofTailAddress: currentAofTailAddress,
currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset,
diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs
index fa063530b5e..41f0e712887 100644
--- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs
+++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs
@@ -13,12 +13,10 @@ namespace Garnet.cluster
internal sealed unsafe class SnapshotIteratorManager
{
public readonly ReplicationSyncManager replicationSyncManager;
- public readonly TimeSpan timeout;
public readonly CancellationToken cancellationToken;
public readonly ILogger logger;
- public MainStoreSnapshotIterator mainStoreSnapshotIterator;
- public ObjectStoreSnapshotIterator objectStoreSnapshotIterator;
+ public StoreSnapshotIterator StoreSnapshotIterator;
// For serialization from LogRecord to DiskLogRecord
SpanByteAndMemory serializationOutput;
@@ -50,9 +48,7 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ca
sessions[i].checkpointCoveredAofAddress = CheckpointCoveredAddress;
}
- mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this);
- if (!replicationSyncManager.ClusterProvider.serverOptions.DisableObjects)
- objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this);
+ StoreSnapshotIterator = new StoreSnapshotIterator(this);
memoryPool = MemoryPool.Shared;
valueObjectSerializer = new(customCommandManager: default);
@@ -76,7 +72,7 @@ public bool IsProgressing()
}
}
- public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion, bool isMainStore)
+ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion)
{
if (cancellationToken.IsCancellationRequested)
{
@@ -92,14 +88,11 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio
if (!replicationSyncManager.IsActive(i))
continue;
sessions[i].InitializeIterationBuffer();
- if (isMainStore)
- sessions[i].currentStoreVersion = targetVersion;
- else
- sessions[i].currentObjectStoreVersion = targetVersion;
+ sessions[i].currentStoreVersion = targetVersion;
}
- logger?.LogTrace("{OnStart} {store} {token} {currentVersion} {targetVersion}",
- nameof(OnStart), isMainStore ? "MAIN STORE" : "OBJECT STORE", checkpointToken, currentVersion, targetVersion);
+ logger?.LogTrace("{OnStart} {token} {currentVersion} {targetVersion}",
+ nameof(OnStart), checkpointToken, currentVersion, targetVersion);
return true;
}
@@ -229,16 +222,15 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long
}
}
- internal sealed unsafe class MainStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) :
+ internal sealed unsafe class StoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) :
IStreamingSnapshotIteratorFunctions
{
- readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager;
long targetVersion;
public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion)
{
this.targetVersion = targetVersion;
- return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: true);
+ return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion);
}
public bool Reader(in TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords)
@@ -246,32 +238,9 @@ public bool Reader(in TSourceLogRecord srcLogRecord, RecordMet
=> snapshotIteratorManager.StringReader(in srcLogRecord, recordMetadata, numberOfRecords);
public void OnException(Exception exception, long numberOfRecords)
- => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(MainStoreSnapshotIterator)}");
+ => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(StoreSnapshotIterator)}");
public void OnStop(bool completed, long numberOfRecords)
=> snapshotIteratorManager.OnStop(completed, numberOfRecords, isMainStore: true, targetVersion);
}
-
- internal sealed unsafe class ObjectStoreSnapshotIterator(SnapshotIteratorManager snapshotIteratorManager) :
- IStreamingSnapshotIteratorFunctions
- {
- readonly SnapshotIteratorManager snapshotIteratorManager = snapshotIteratorManager;
- long targetVersion;
-
- public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion)
- {
- this.targetVersion = targetVersion;
- return snapshotIteratorManager.OnStart(checkpointToken, currentVersion, targetVersion, isMainStore: false);
- }
-
- public bool Reader(in TSourceLogRecord srcLogRecord, RecordMetadata recordMetadata, long numberOfRecords)
- where TSourceLogRecord : ISourceLogRecord
- => snapshotIteratorManager.ObjectReader(in srcLogRecord, recordMetadata, numberOfRecords);
-
- public void OnException(Exception exception, long numberOfRecords)
- => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(ObjectStoreSnapshotIterator)}");
-
- public void OnStop(bool completed, long numberOfRecords)
- => snapshotIteratorManager.OnStop(completed, numberOfRecords, isMainStore: false, targetVersion);
- }
}
\ No newline at end of file
diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs
index e57e0f37b33..f04db715eaa 100644
--- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs
+++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs
@@ -155,8 +155,6 @@ public async Task ReplicationSyncDriver(ReplicaSyncSession repli
async Task MainStreamingSnapshotDriver()
{
// Parameters for sync operation
- var disableObjects = ClusterProvider.serverOptions.DisableObjects;
-
try
{
// Lock to avoid the addition of new replica sync sessions while sync is in progress
@@ -242,7 +240,6 @@ async Task PrepareForSync()
// Set store version to operate on
Sessions[i].currentStoreVersion = ClusterProvider.storeWrapper.store.CurrentVersion;
- Sessions[i].currentObjectStoreVersion = disableObjects ? -1 : ClusterProvider.storeWrapper.objectStore.CurrentVersion;
// If checkpoint is not needed mark this sync session as complete
// to avoid waiting for other replicas which may need to receive the latest checkpoint
@@ -280,47 +277,50 @@ async Task TakeStreamingCheckpoint()
// Iterate through main store
var mainStoreCheckpointTask = ClusterProvider.storeWrapper.store.
- TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator);
+ TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, cancellationToken: cts.Token, streamingSnapshotIteratorFunctions: manager.StoreSnapshotIterator);
var result = await WaitOrDie(checkpointTask: mainStoreCheckpointTask, iteratorManager: manager);
if (!result.success)
- throw new InvalidOperationException("Main store checkpoint stream failed!");
-
- if (!ClusterProvider.serverOptions.DisableObjects)
- {
- // Iterate through object store
- var objectStoreCheckpointTask = ClusterProvider.storeWrapper.objectStore.
- TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator);
- result = await WaitOrDie(checkpointTask: objectStoreCheckpointTask, iteratorManager: manager);
- if (!result.success)
- throw new InvalidOperationException("Object store checkpoint stream failed!");
- }
+ throw new GarnetException("Main store checkpoint stream failed!");
// Note: We do not truncate the AOF here as this was just a "virtual" checkpoint
-
+ // WaitOrDie is needed here to check if streaming checkpoint is making progress.
+ // We cannot use a timeout on the cancellationToken because we don't know in total how long the streaming checkpoint will take
async ValueTask<(bool success, Guid token)> WaitOrDie(ValueTask<(bool success, Guid token)> checkpointTask, SnapshotIteratorManager iteratorManager)
{
- var timeout = replicaSyncTimeout;
- var delay = TimeSpan.FromSeconds(1);
- while (true)
+ try
{
- // Check if cancellation requested
- cts.Token.ThrowIfCancellationRequested();
+ var timeout = replicaSyncTimeout;
+ var delay = TimeSpan.FromSeconds(1);
+ while (true)
+ {
+ // Check if cancellation requested
+ cts.Token.ThrowIfCancellationRequested();
- // Wait for stream sync to make some progress
- await Task.Delay(delay);
+ // Wait for stream sync to make some progress
+ await Task.Delay(delay);
- // Check if checkpoint has completed
- if (checkpointTask.IsCompleted)
- return await checkpointTask;
+ // Check if checkpoint has completed
+ if (checkpointTask.IsCompleted)
+ return await checkpointTask;
- // Check if we made some progress
- timeout = !manager.IsProgressing() ? timeout.Subtract(delay) : replicaSyncTimeout;
+ // Check if we made some progress
+ timeout = !manager.IsProgressing() ? timeout.Subtract(delay) : replicaSyncTimeout;
- // Throw timeout equals to zero
- if (timeout.TotalSeconds <= 0)
- throw new TimeoutException("Streaming snapshot checkpoint timed out");
+ // Throw timeout equals to zero
+ if (timeout.TotalSeconds <= 0)
+ {
+ cts.Cancel();
+ throw new TimeoutException("Streaming snapshot checkpoint timed out");
+ }
+ }
+ }
+ catch (Exception ex)
+ {
+ logger?.LogError(ex, "{method} faulted", nameof(WaitOrDie));
+ cts.Cancel();
}
+ return (false, default);
}
}
}
diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs
index 54ecd50a934..20b64d1e447 100644
--- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs
+++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs
@@ -61,37 +61,25 @@ public bool ValidateMetadata(
CheckpointEntry localEntry,
out long index_size,
out LogFileInfo hlog_size,
- out long obj_index_size,
- out LogFileInfo obj_hlog_size,
- out bool skipLocalMainStoreCheckpoint,
- out bool skipLocalObjectStoreCheckpoint)
+ out bool skipLocalMainStoreCheckpoint)
{
hlog_size = default;
- obj_hlog_size = default;
index_size = -1L;
- obj_index_size = -1L;
// Local and remote checkpoints are of same history if both of the following hold
// 1. There is a checkpoint available at remote node
// 2. Remote and local checkpoints contain the same PrimaryReplId
var sameMainStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.storePrimaryReplId) && replicaCheckpointEntry.metadata.storePrimaryReplId.Equals(localEntry.metadata.storePrimaryReplId);
- var sameObjectStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.objectStorePrimaryReplId) && replicaCheckpointEntry.metadata.objectStorePrimaryReplId.Equals(localEntry.metadata.objectStorePrimaryReplId);
// We will not send the latest local checkpoint if any of the following hold
// 1. Local node does not have any checkpoints
// 2. Local checkpoint is of same version and history as the remote checkpoint
skipLocalMainStoreCheckpoint = localEntry.metadata.storeHlogToken == default || (sameMainStoreCheckpointHistory && localEntry.metadata.storeVersion == replicaCheckpointEntry.metadata.storeVersion);
- skipLocalObjectStoreCheckpoint = clusterProvider.serverOptions.DisableObjects || localEntry.metadata.objectStoreHlogToken == default || (sameObjectStoreCheckpointHistory && localEntry.metadata.objectStoreVersion == replicaCheckpointEntry.metadata.objectStoreVersion);
// Acquire metadata for main store
// If failed then this checkpoint is not usable because it is corrupted
if (!skipLocalMainStoreCheckpoint && !clusterProvider.replicationManager.TryAcquireSettledMetadataForMainStore(localEntry, out hlog_size, out index_size))
return false;
- // Acquire metadata for object store
- // If failed then this checkpoint is not usable because it is corrupted
- if (!skipLocalObjectStoreCheckpoint && !clusterProvider.replicationManager.TryAcquireSettledMetadataForObjectStore(localEntry, out obj_hlog_size, out obj_index_size))
- return false;
-
return true;
}
@@ -101,8 +89,7 @@ public bool ValidateMetadata(
public async Task SendCheckpoint()
{
errorMsg = default;
- var storeCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main);
- var objectStoreCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object);
+ var storeCkptManager = clusterProvider.GetReplicationLogCheckpointManager();
var current = clusterProvider.clusterManager.CurrentConfig;
var (address, port) = current.GetWorkerAddressFromNodeId(replicaNodeId);
@@ -126,8 +113,8 @@ public async Task SendCheckpoint()
try
{
- logger?.LogInformation("Replica replicaId:{replicaId} requesting checkpoint replicaStoreVersion:{replicaStoreVersion} replicaObjectStoreVersion:{replicaObjectStoreVersion}",
- replicaNodeId, replicaCheckpointEntry.metadata.storeVersion, replicaCheckpointEntry.metadata.objectStoreVersion);
+ logger?.LogInformation("Replica replicaId:{replicaId} requesting checkpoint replicaStoreVersion:{replicaStoreVersion}",
+ replicaNodeId, replicaCheckpointEntry.metadata.storeVersion);
logger?.LogInformation("Attempting to acquire checkpoint");
(localEntry, aofSyncTaskInfo) = await AcquireCheckpointEntry();
@@ -136,13 +123,10 @@ public async Task SendCheckpoint()
gcs.Connect((int)storeWrapper.serverOptions.ReplicaSyncTimeout.TotalMilliseconds);
long index_size = -1;
- long obj_index_size = -1;
var hlog_size = default(LogFileInfo);
- var obj_hlog_size = default(LogFileInfo);
var skipLocalMainStoreCheckpoint = false;
- var skipLocalObjectStoreCheckpoint = false;
var retryCount = validateMetadataMaxRetryCount;
- while (!ValidateMetadata(localEntry, out index_size, out hlog_size, out obj_index_size, out obj_hlog_size, out skipLocalMainStoreCheckpoint, out skipLocalObjectStoreCheckpoint))
+ while (!ValidateMetadata(localEntry, out index_size, out hlog_size, out skipLocalMainStoreCheckpoint))
{
logger?.LogError("Failed to validate metadata. Retrying....");
await Task.Yield();
@@ -177,50 +161,10 @@ public async Task SendCheckpoint()
await SendCheckpointMetadata(gcs, storeCkptManager, CheckpointFileType.STORE_SNAPSHOT, localEntry.metadata.storeHlogToken);
}
- if (!skipLocalObjectStoreCheckpoint)
- {
- logger?.LogInformation("Sending object store checkpoint {version} {objectStoreHlogToken} {objectStoreIndexToken} to replica", localEntry.metadata.objectStoreVersion, localEntry.metadata.objectStoreHlogToken, localEntry.metadata.objectStoreIndexToken);
-
- // 1. send hlog file segments
- if (clusterProvider.serverOptions.EnableStorageTier && obj_hlog_size.hybridLogFileEndAddress > 24)
- {
- //send object hlog file segments
- await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_HLOG, obj_hlog_size.hybridLogFileStartAddress, obj_hlog_size.hybridLogFileEndAddress);
-
- var hlogSegmentCount = ((obj_hlog_size.hybridLogFileEndAddress - obj_hlog_size.hybridLogFileStartAddress) >> clusterProvider.serverOptions.ObjectStoreSegmentSizeBits()) + 1;
- await SendObjectFiles(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_HLOG_OBJ, (int)hlogSegmentCount);
- }
-
- // 2. Send object store snapshot files
- if (obj_hlog_size.snapshotFileEndAddress > 24)
- {
- //send snapshot file segments
- await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_SNAPSHOT, 0, obj_hlog_size.snapshotFileEndAddress);
-
- //send snapshot.obj file segments
- var snapshotSegmentCount = (obj_hlog_size.snapshotFileEndAddress >> clusterProvider.serverOptions.ObjectStoreSegmentSizeBits()) + 1;
- await SendObjectFiles(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_SNAPSHOT_OBJ, (int)snapshotSegmentCount);
- }
-
- // 3. Send object store index file segments
- if (obj_index_size > 0)
- await SendFileSegments(gcs, localEntry.metadata.objectStoreIndexToken, CheckpointFileType.OBJ_STORE_INDEX, 0, obj_index_size);
-
- // 4. Send object store delta file segments
- var obj_dlog_size = obj_hlog_size.deltaLogTailAddress;
- if (obj_dlog_size > 0)
- await SendFileSegments(gcs, localEntry.metadata.objectStoreHlogToken, CheckpointFileType.OBJ_STORE_DLOG, 0, obj_dlog_size);
-
- // 5. Send object store index metadata
- await SendCheckpointMetadata(gcs, objectStoreCkptManager, CheckpointFileType.OBJ_STORE_INDEX, localEntry.metadata.objectStoreIndexToken);
-
- // 6. Send object store snapshot metadata
- await SendCheckpointMetadata(gcs, objectStoreCkptManager, CheckpointFileType.OBJ_STORE_SNAPSHOT, localEntry.metadata.objectStoreHlogToken);
- }
#endregion
#region startAofSync
- var recoverFromRemote = !skipLocalMainStoreCheckpoint || !skipLocalObjectStoreCheckpoint;
+ var recoverFromRemote = !skipLocalMainStoreCheckpoint;
var replayAOF = false;
var checkpointAofBeginAddress = localEntry.GetMinAofCoveredAddress();
var beginAddress = checkpointAofBeginAddress;
@@ -266,8 +210,7 @@ public async Task SendCheckpoint()
}
var sameMainStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.storePrimaryReplId) && replicaCheckpointEntry.metadata.storePrimaryReplId.Equals(localEntry.metadata.storePrimaryReplId);
- var sameObjectStoreCheckpointHistory = !string.IsNullOrEmpty(replicaCheckpointEntry.metadata.objectStorePrimaryReplId) && replicaCheckpointEntry.metadata.objectStorePrimaryReplId.Equals(localEntry.metadata.objectStorePrimaryReplId);
- if (!sameMainStoreCheckpointHistory || !sameObjectStoreCheckpointHistory)
+ if (!sameMainStoreCheckpointHistory)
{
// If we are not in the same checkpoint history, we need to stream the AOF from the primary's beginning address
checkpointAofBeginAddress = beginAddress;
@@ -281,7 +224,6 @@ public async Task SendCheckpoint()
// Make replica replayAOF if needed and replay from provided beginAddress to RecoveredReplication Address
var resp = await gcs.ExecuteBeginReplicaRecover(
!skipLocalMainStoreCheckpoint,
- !skipLocalObjectStoreCheckpoint,
replayAOF,
clusterProvider.replicationManager.PrimaryReplId,
localEntry.ToByteArray(),
@@ -395,7 +337,7 @@ public async Task SendCheckpoint()
// If there is possible AOF data loss and we need to take an on-demand checkpoint,
// then we should take the checkpoint before we register the sync task, because
// TryAddReplicationTask is guaranteed to return true in this scenario.
- var validMetadata = ValidateMetadata(cEntry, out _, out _, out _, out _, out _, out _);
+ var validMetadata = ValidateMetadata(cEntry, out _, out _, out _);
if (clusterProvider.serverOptions.OnDemandCheckpoint &&
(startAofAddress < clusterProvider.replicationManager.AofTruncatedUntil || !validMetadata))
{
@@ -443,11 +385,9 @@ private async Task SendCheckpointMetadata(GarnetClientSession gcs, GarnetCluster
switch (fileType)
{
case CheckpointFileType.STORE_SNAPSHOT:
- case CheckpointFileType.OBJ_STORE_SNAPSHOT:
checkpointMetadata = ckptManager.GetLogCheckpointMetadata(fileToken, null, true, -1);
break;
case CheckpointFileType.STORE_INDEX:
- case CheckpointFileType.OBJ_STORE_INDEX:
checkpointMetadata = ckptManager.GetIndexCheckpointMetadata(fileToken);
break;
}
diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs
index 01b577e5027..b5c50462e6e 100644
--- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs
+++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs
@@ -4,9 +4,11 @@
using System;
using System.Net;
using System.Text;
+using System.Threading;
using System.Threading.Tasks;
using Garnet.client;
using Garnet.cluster.Server.Replication;
+using Garnet.common;
using Microsoft.Extensions.Logging;
namespace Garnet.cluster
@@ -56,8 +58,8 @@ public bool TryReplicateDisklessSync(
async Task TryBeginReplicaSync(bool downgradeLock)
{
var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync;
- var disableObjects = clusterProvider.serverOptions.DisableObjects;
GarnetClientSession gcs = null;
+ resetHandler ??= new CancellationTokenSource();
try
{
if (!clusterProvider.serverOptions.EnableFastCommit)
@@ -113,13 +115,17 @@ async Task TryBeginReplicaSync(bool downgradeLock)
originNodeId: current.LocalNodeId,
currentPrimaryReplId: PrimaryReplId,
currentStoreVersion: storeWrapper.store.CurrentVersion,
- currentObjectStoreVersion: disableObjects ? -1 : storeWrapper.objectStore.CurrentVersion,
currentAofBeginAddress: storeWrapper.appendOnlyFile.BeginAddress,
currentAofTailAddress: storeWrapper.appendOnlyFile.TailAddress,
currentReplicationOffset: ReplicationOffset,
checkpointEntry: checkpointEntry);
- var resp = await gcs.ExecuteAttachSync(syncMetadata.ToByteArray()).ConfigureAwait(false);
+ using var linkedCts = CancellationTokenSource.CreateLinkedTokenSource(ctsRepManager.Token, resetHandler.Token);
+
+ // Exception injection point for testing cluster reset during diskless replication
+ await ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Replication_InProgress_During_Diskless_Replica_Attach_Sync).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false);
+
+ var resp = await gcs.ExecuteAttachSync(syncMetadata.ToByteArray()).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false);
}
catch (Exception ex)
{
@@ -144,6 +150,11 @@ async Task TryBeginReplicaSync(bool downgradeLock)
}
gcs?.Dispose();
recvCheckpointHandler?.Dispose();
+ if (!resetHandler.TryReset())
+ {
+ resetHandler.Dispose();
+ resetHandler = new CancellationTokenSource();
+ }
}
return null;
}
@@ -171,8 +182,6 @@ public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata, out ReadOnl
// Set DB version
storeWrapper.store.SetVersion(primarySyncMetadata.currentStoreVersion);
- if (!clusterProvider.serverOptions.DisableObjects)
- storeWrapper.objectStore.SetVersion(primarySyncMetadata.currentObjectStoreVersion);
// Update replicationId to mark any subsequent checkpoints as part of this history
logger?.LogInformation("Updating ReplicationId");
diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs
index 36ef01d4520..3bb0d31dd72 100644
--- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs
+++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs
@@ -6,10 +6,11 @@
using System.IO;
using System.Net;
using System.Text;
+using System.Threading;
using System.Threading.Tasks;
using Garnet.client;
using Garnet.cluster.Server.Replication;
-using Garnet.server;
+using Garnet.common;
using Microsoft.Extensions.Logging;
using Tsavorite.core;
@@ -72,6 +73,7 @@ async Task ReplicaSyncAttachTask(bool downgradeLock)
{
Debug.Assert(IsRecovering);
GarnetClientSession gcs = null;
+ resetHandler ??= new CancellationTokenSource();
try
{
// Immediately try to connect to a primary, so we FAIL
@@ -139,12 +141,16 @@ async Task ReplicaSyncAttachTask(bool downgradeLock)
// 4. Replica responds with aofStartAddress sync
// 5. Primary will initiate aof sync task
// 6. Primary releases checkpoint
+ using var linkedCts = CancellationTokenSource.CreateLinkedTokenSource(ctsRepManager.Token, resetHandler.Token);
+
+ // Exception injection point for testing cluster reset during disk-based replication
+ await ExceptionInjectionHelper.WaitOnSet(ExceptionInjectionType.Replication_InProgress_During_DiskBased_Replica_Attach_Sync).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false);
var resp = await gcs.ExecuteReplicaSync(
nodeId,
PrimaryReplId,
cEntry.ToByteArray(),
storeWrapper.appendOnlyFile.BeginAddress,
- storeWrapper.appendOnlyFile.TailAddress).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, ctsRepManager.Token).ConfigureAwait(false);
+ storeWrapper.appendOnlyFile.TailAddress).WaitAsync(storeWrapper.serverOptions.ReplicaAttachTimeout, linkedCts.Token).ConfigureAwait(false);
}
catch (Exception ex)
{
@@ -167,6 +173,11 @@ async Task ReplicaSyncAttachTask(bool downgradeLock)
}
recvCheckpointHandler?.Dispose();
gcs?.Dispose();
+ if (!resetHandler.TryReset())
+ {
+ resetHandler.Dispose();
+ resetHandler = new CancellationTokenSource();
+ }
}
return null;
}
@@ -185,20 +196,16 @@ public void ProcessCheckpointMetadata(Guid fileToken, CheckpointFileType fileTyp
var ckptManager = fileType switch
{
CheckpointFileType.STORE_SNAPSHOT or
- CheckpointFileType.STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main),
- CheckpointFileType.OBJ_STORE_SNAPSHOT or
- CheckpointFileType.OBJ_STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object),
+ CheckpointFileType.STORE_INDEX => clusterProvider.GetReplicationLogCheckpointManager(),
_ => throw new Exception($"Invalid checkpoint filetype {fileType}"),
};
switch (fileType)
{
case CheckpointFileType.STORE_SNAPSHOT:
- case CheckpointFileType.OBJ_STORE_SNAPSHOT:
ckptManager.CommitLogCheckpointSendFromPrimary(fileToken, checkpointMetadata);
break;
case CheckpointFileType.STORE_INDEX:
- case CheckpointFileType.OBJ_STORE_INDEX:
ckptManager.CommitIndexCheckpoint(fileToken, checkpointMetadata);
break;
default:
@@ -217,9 +224,7 @@ public static bool ShouldInitialize(CheckpointFileType type)
return type switch
{
CheckpointFileType.STORE_HLOG or
- CheckpointFileType.STORE_SNAPSHOT or
- CheckpointFileType.OBJ_STORE_HLOG or
- CheckpointFileType.OBJ_STORE_SNAPSHOT
+ CheckpointFileType.STORE_SNAPSHOT
=> true,
_ => false,
};
@@ -236,9 +241,7 @@ public IDevice GetInitializedSegmentFileDevice(Guid token, CheckpointFileType ty
var device = type switch
{
CheckpointFileType.STORE_HLOG => GetStoreHLogDevice(),
- CheckpointFileType.OBJ_STORE_HLOG => GetObjectStoreHLogDevice(false),//TODO: return device for object store hlog
- CheckpointFileType.OBJ_STORE_HLOG_OBJ => GetObjectStoreHLogDevice(true),
- _ => clusterProvider.GetReplicationLogCheckpointManager(type.ToStoreType()).GetDevice(type, token),
+ _ => clusterProvider.GetReplicationLogCheckpointManager().GetDevice(type, token),
};
if (ShouldInitialize(type))
@@ -257,35 +260,21 @@ IDevice GetStoreHLogDevice()
}
return null;
}
-
- IDevice GetObjectStoreHLogDevice(bool obj)
- {
- var opts = clusterProvider.serverOptions;
- if (opts.EnableStorageTier)
- {
- var LogDir = opts.LogDir;
- if (LogDir is null or "") LogDir = Directory.GetCurrentDirectory();
- var logFactory = opts.GetInitializedDeviceFactory(LogDir);
- return obj ? logFactory.Get(new FileDescriptor("ObjectStore", "hlog.obj")) : logFactory.Get(new FileDescriptor("ObjectStore", "hlog"));
- }
- return null;
- }
}
///
/// Process request from primary to start recovery process from the retrieved checkpoint.
///
///
- ///
///
///
///
///
///
+ ///
///
public long BeginReplicaRecover(
bool recoverMainStoreFromToken,
- bool recoverObjectStoreFromToken,
bool replayAOF,
string primaryReplicationId,
CheckpointEntry remoteCheckpoint,
@@ -298,19 +287,14 @@ public long BeginReplicaRecover(
errorMessage = [];
UpdateLastPrimarySyncTime();
- logger?.LogInformation("Replica Recover MainStore: {storeVersion}>[{sIndexToken} {sHlogToken}]" +
- "\nObjectStore: {objectStoreVersion}>[{oIndexToken} {oHlogToken}]",
+ logger?.LogInformation("Replica Recover Store: {storeVersion}>[{sIndexToken} {sHlogToken}]",
remoteCheckpoint.metadata.storeVersion,
remoteCheckpoint.metadata.storeIndexToken,
- remoteCheckpoint.metadata.storeHlogToken,
- remoteCheckpoint.metadata.objectStoreVersion,
- remoteCheckpoint.metadata.objectStoreIndexToken,
- remoteCheckpoint.metadata.objectStoreHlogToken);
+ remoteCheckpoint.metadata.storeHlogToken);
storeWrapper.RecoverCheckpoint(
replicaRecover: true,
recoverMainStoreFromToken,
- recoverObjectStoreFromToken,
remoteCheckpoint.metadata);
if (replayAOF)
@@ -333,12 +317,6 @@ public long BeginReplicaRecover(
cEntry.metadata.storeHlogToken = remoteCheckpoint.metadata.storeHlogToken;
}
- // If checkpoint for object store was send add its token here in preparation for purge later on
- if (recoverObjectStoreFromToken)
- {
- cEntry.metadata.objectStoreIndexToken = remoteCheckpoint.metadata.objectStoreIndexToken;
- cEntry.metadata.objectStoreHlogToken = remoteCheckpoint.metadata.objectStoreHlogToken;
- }
checkpointStore.PurgeAllCheckpointsExceptEntry(cEntry);
// Initialize in-memory checkpoint store and delete outdated checkpoint entries
diff --git a/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs b/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs
index e22db3ad2b1..1bfceec27ef 100644
--- a/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs
+++ b/libs/cluster/Server/Replication/ReplicationCheckpointManagement.cs
@@ -45,29 +45,6 @@ public bool TryAcquireSettledMetadataForMainStore(CheckpointEntry entry, out Log
}
}
- ///
- /// Keep trying to acquire object store metadata until it settles
- ///
- /// CheckpointEntry to retrieve metadata for
- /// LogFileInfo to return
- /// Index size in bytes to return
- public bool TryAcquireSettledMetadataForObjectStore(CheckpointEntry entry, out LogFileInfo hlog_size, out long index_size)
- {
- hlog_size = default;
- index_size = -1;
- try
- {
- hlog_size = storeWrapper.objectStore.GetLogFileSize(entry.metadata.objectStoreHlogToken);
- index_size = storeWrapper.objectStore.GetIndexFileSize(entry.metadata.objectStoreIndexToken);
- return true;
- }
- catch (Exception ex)
- {
- logger?.LogError(ex, "Waiting for object store metadata to settle");
- return false;
- }
- }
-
///
/// Add new checkpoint entry to the in-memory store
///
@@ -90,10 +67,8 @@ public string GetLatestCheckpointFromDiskInfo()
#endregion
public long StoreCurrentSafeAofAddress => clusterProvider.storeWrapper.StoreCheckpointManager.CurrentSafeAofAddress;
- public long ObjectStoreCurrentSafeAofAddress => clusterProvider.serverOptions.DisableObjects ? -1 : clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentSafeAofAddress;
public long StoreRecoveredSafeAofTailAddress => clusterProvider.storeWrapper.StoreCheckpointManager.RecoveredSafeAofAddress;
- public long ObjectStoreRecoveredSafeAofTailAddress => clusterProvider.serverOptions.DisableObjects ? -1 : clusterProvider.storeWrapper.ObjectStoreCheckpointManager.RecoveredSafeAofAddress;
///
/// Update current aof address for pending commit.
@@ -103,8 +78,6 @@ public string GetLatestCheckpointFromDiskInfo()
public void UpdateCommitSafeAofAddress(long safeAofTailAddress)
{
clusterProvider.storeWrapper.StoreCheckpointManager.CurrentSafeAofAddress = safeAofTailAddress;
- if (!clusterProvider.serverOptions.DisableObjects)
- clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentSafeAofAddress = safeAofTailAddress;
}
///
@@ -114,8 +87,6 @@ public void UpdateCommitSafeAofAddress(long safeAofTailAddress)
public void SetPrimaryReplicationId()
{
clusterProvider.storeWrapper.StoreCheckpointManager.CurrentHistoryId = PrimaryReplId;
- if (!clusterProvider.serverOptions.DisableObjects)
- clusterProvider.storeWrapper.ObjectStoreCheckpointManager.CurrentHistoryId = PrimaryReplId;
}
}
}
\ No newline at end of file
diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs
index 7eecb80878e..a840ce00a8e 100644
--- a/libs/cluster/Server/Replication/ReplicationManager.cs
+++ b/libs/cluster/Server/Replication/ReplicationManager.cs
@@ -25,6 +25,7 @@ internal sealed partial class ReplicationManager : IDisposable
readonly CheckpointStore checkpointStore;
readonly ReplicationSyncManager replicationSyncManager;
readonly CancellationTokenSource ctsRepManager = new();
+ CancellationTokenSource resetHandler = new();
readonly int pageSizeBits;
@@ -86,29 +87,14 @@ public long ReplicationOffset2
public RecoveryStatus currentRecoveryStatus;
[MethodImpl(MethodImplOptions.AggressiveInlining)]
- public GarnetClusterCheckpointManager GetCkptManager(StoreType storeType)
- {
- return storeType switch
- {
- StoreType.Main => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager,
- StoreType.Object => (GarnetClusterCheckpointManager)storeWrapper.objectStore?.CheckpointManager,
- _ => throw new Exception($"GetCkptManager: unexpected state {storeType}")
- };
- }
+ public GarnetClusterCheckpointManager GetCkptManager()
+ => (GarnetClusterCheckpointManager)storeWrapper.store.CheckpointManager;
public long GetRecoveredSafeAofAddress()
- {
- var storeAofAddress = clusterProvider.replicationManager.GetCkptManager(StoreType.Main).RecoveredSafeAofAddress;
- var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager(StoreType.Object).RecoveredSafeAofAddress;
- return Math.Min(storeAofAddress, objectStoreAofAddress);
- }
+ => clusterProvider.replicationManager.GetCkptManager().RecoveredSafeAofAddress;
public long GetCurrentSafeAofAddress()
- {
- var storeAofAddress = clusterProvider.replicationManager.GetCkptManager(StoreType.Main).CurrentSafeAofAddress;
- var objectStoreAofAddress = clusterProvider.serverOptions.DisableObjects ? long.MaxValue : clusterProvider.replicationManager.GetCkptManager(StoreType.Object).CurrentSafeAofAddress;
- return Math.Min(storeAofAddress, objectStoreAofAddress);
- }
+ => clusterProvider.replicationManager.GetCkptManager().CurrentSafeAofAddress;
public ReplicationManager(ClusterProvider clusterProvider, ILogger logger = null)
{
@@ -129,13 +115,8 @@ public ReplicationManager(ClusterProvider clusterProvider, ILogger logger = null
ReplicationOffset = 0;
// Set the appendOnlyFile field for all stores
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).checkpointVersionShiftStart = CheckpointVersionShiftStart;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main).checkpointVersionShiftEnd = CheckpointVersionShiftEnd;
- if (storeWrapper.objectStore != null)
- {
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).checkpointVersionShiftStart = CheckpointVersionShiftStart;
- clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object).checkpointVersionShiftEnd = CheckpointVersionShiftEnd;
- }
+ clusterProvider.GetReplicationLogCheckpointManager().checkpointVersionShiftStart = CheckpointVersionShiftStart;
+ clusterProvider.GetReplicationLogCheckpointManager().checkpointVersionShiftEnd = CheckpointVersionShiftEnd;
// If this node starts as replica, it cannot serve requests until it is connected to primary
if (clusterProvider.clusterManager.CurrentConfig.LocalNodeRole == NodeRole.REPLICA && clusterProvider.serverOptions.Recover && !BeginRecovery(RecoveryStatus.InitializeRecover, upgradeLock: false))
@@ -454,6 +435,20 @@ public void EndRecovery(RecoveryStatus nextRecoveryStatus, bool downgradeLock)
}
}
+ public void ResetRecovery()
+ {
+ switch (currentRecoveryStatus)
+ {
+ case RecoveryStatus.ClusterReplicate:
+ case RecoveryStatus.ClusterFailover:
+ case RecoveryStatus.ReplicaOfNoOne:
+ case RecoveryStatus.CheckpointRecoveredAtReplica:
+ case RecoveryStatus.InitializeRecover:
+ resetHandler.Cancel();
+ break;
+ }
+ }
+
public void Dispose()
{
_disposed = true;
@@ -470,6 +465,8 @@ public void Dispose()
replicaReplayTaskCts.Dispose();
ctsRepManager.Cancel();
ctsRepManager.Dispose();
+ resetHandler.Cancel();
+ resetHandler.Dispose();
aofTaskStore.Dispose();
aofProcessor?.Dispose();
networkPool?.Dispose();
diff --git a/libs/cluster/Server/Replication/SyncMetadata.cs b/libs/cluster/Server/Replication/SyncMetadata.cs
index 927f600e123..c85853ecf6d 100644
--- a/libs/cluster/Server/Replication/SyncMetadata.cs
+++ b/libs/cluster/Server/Replication/SyncMetadata.cs
@@ -26,7 +26,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
"originNodeId:{originNodeId}\n" +
"currentPrimaryReplId:{currentPrimaryReplId}\n" +
"currentStoreVersion:{currentStoreVersion}\n" +
- "currentObjectStoreVersion:{currentObjectStoreVersion}\n" +
"currentAofBeginAddress:{currentAofBeginAddress}\n" +
"currentAofTailAddress:{currentAofTailAddress}\n" +
"currentReplicationOffset:{currentReplicationOffset}\n" +
@@ -37,7 +36,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
syncMetadata.originNodeId,
syncMetadata.currentPrimaryReplId,
syncMetadata.currentStoreVersion,
- syncMetadata.currentObjectStoreVersion,
syncMetadata.currentAofBeginAddress,
syncMetadata.currentAofTailAddress,
syncMetadata.currentReplicationOffset,
@@ -63,7 +61,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
"originNodeId:{originNodeId}\n" +
"currentPrimaryReplId:{currentPrimaryReplId}\n" +
"currentStoreVersion:{currentStoreVersion}\n" +
- "currentObjectStoreVersion:{currentObjectStoreVersion}\n" +
"currentAofBeginAddress:{currentAofBeginAddress}\n" +
"currentAofTailAddress:{currentAofTailAddress}\n" +
"currentReplicationOffset:{currentReplicationOffset}\n" +
@@ -74,7 +71,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
"recoverOriginNodeId:{originNodeId}\n" +
"recoverCurrentPrimaryReplId:{currentPrimaryReplId}\n" +
"recoverCurrentStoreVersion:{currentStoreVersion}\n" +
- "recoverCurrentObjectStoreVersion:{currentObjectStoreVersion}\n" +
"recoverCurrentAofBeginAddress:{currentAofBeginAddress}\n" +
"recoverCurrentAofTailAddress:{currentAofTailAddress}\n" +
"recoverCurrentReplicationOffset:{currentReplicationOffset}\n" +
@@ -85,7 +81,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
origin.originNodeId,
origin.currentPrimaryReplId,
origin.currentStoreVersion,
- origin.currentObjectStoreVersion,
origin.currentAofBeginAddress,
origin.currentAofTailAddress,
origin.currentReplicationOffset,
@@ -95,7 +90,6 @@ public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string m
local.originNodeId,
local.currentPrimaryReplId,
local.currentStoreVersion,
- local.currentObjectStoreVersion,
local.currentAofBeginAddress,
local.currentAofTailAddress,
local.currentReplicationOffset,
@@ -109,7 +103,6 @@ internal sealed class SyncMetadata(
string originNodeId,
string currentPrimaryReplId,
long currentStoreVersion,
- long currentObjectStoreVersion,
long currentAofBeginAddress,
long currentAofTailAddress,
long currentReplicationOffset,
@@ -120,7 +113,6 @@ internal sealed class SyncMetadata(
public readonly string originNodeId = originNodeId;
public readonly string currentPrimaryReplId = currentPrimaryReplId;
public readonly long currentStoreVersion = currentStoreVersion;
- public readonly long currentObjectStoreVersion = currentObjectStoreVersion;
public readonly long currentAofBeginAddress = currentAofBeginAddress;
public readonly long currentAofTailAddress = currentAofTailAddress;
public readonly long currentReplicationOffset = currentReplicationOffset;
@@ -137,7 +129,6 @@ public byte[] ToByteArray()
writer.Write(currentPrimaryReplId);
writer.Write(currentStoreVersion);
- writer.Write(currentObjectStoreVersion);
writer.Write(currentAofBeginAddress);
writer.Write(currentAofTailAddress);
@@ -168,7 +159,6 @@ public static SyncMetadata FromByteArray(byte[] serialized)
originNodeId: reader.ReadString(),
currentPrimaryReplId: reader.ReadString(),
currentStoreVersion: reader.ReadInt64(),
- currentObjectStoreVersion: reader.ReadInt64(),
currentAofBeginAddress: reader.ReadInt64(),
currentAofTailAddress: reader.ReadInt64(),
currentReplicationOffset: reader.ReadInt64(),
diff --git a/libs/cluster/Session/ClusterCommands.cs b/libs/cluster/Session/ClusterCommands.cs
index 104e05144b7..c9dd8c4b13c 100644
--- a/libs/cluster/Session/ClusterCommands.cs
+++ b/libs/cluster/Session/ClusterCommands.cs
@@ -15,7 +15,7 @@ internal sealed unsafe partial class ClusterSession : IClusterSession
{
ClusterConfig lastSentConfig;
- private int CountKeysInSessionStore(int slot)
+ private int CountKeysInSlot(int slot)
{
ClusterKeyIterationFunctions.MainStoreCountKeys iterFuncs = new(slot);
var cursor = 0L;
@@ -23,33 +23,12 @@ private int CountKeysInSessionStore(int slot)
return iterFuncs.KeyCount;
}
- private int CountKeysInObjectStore(int slot)
- {
- if (!clusterProvider.serverOptions.DisableObjects)
- {
- ClusterKeyIterationFunctions.ObjectStoreCountKeys iterFuncs = new(slot);
- var cursor = 0L;
- _ = basicGarnetApi.IterateObjectStore(ref iterFuncs, ref cursor);
- return iterFuncs.KeyCount;
- }
- return 0;
- }
-
- private int CountKeysInSlot(int slot) => CountKeysInSessionStore(slot) + CountKeysInObjectStore(slot);
-
private List GetKeysInSlot(int slot, int keyCount)
{
List keys = [];
ClusterKeyIterationFunctions.MainStoreGetKeysInSlot mainIterFuncs = new(keys, slot, keyCount);
var cursor = 0L;
_ = basicGarnetApi.IterateMainStore(ref mainIterFuncs, ref cursor);
-
- if (!clusterProvider.serverOptions.DisableObjects)
- {
- ClusterKeyIterationFunctions.ObjectStoreGetKeysInSlot objectIterFuncs = new(keys, slot);
- var objectCursor = 0L;
- _ = basicGarnetApi.IterateObjectStore(ref objectIterFuncs, ref objectCursor);
- }
return keys;
}
diff --git a/libs/cluster/Session/ClusterSession.cs b/libs/cluster/Session/ClusterSession.cs
index 642ba634791..ee67c83175b 100644
--- a/libs/cluster/Session/ClusterSession.cs
+++ b/libs/cluster/Session/ClusterSession.cs
@@ -13,10 +13,13 @@
namespace Garnet.cluster
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
ObjectAllocator>>>;
internal sealed unsafe partial class ClusterSession : IClusterSession
diff --git a/libs/cluster/Session/RespClusterMigrateCommands.cs b/libs/cluster/Session/RespClusterMigrateCommands.cs
index 55a96a0b608..7d1f9b63ce5 100644
--- a/libs/cluster/Session/RespClusterMigrateCommands.cs
+++ b/libs/cluster/Session/RespClusterMigrateCommands.cs
@@ -13,11 +13,14 @@
namespace Garnet.cluster
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
- BasicContext,
- ObjectAllocator>>>;
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
+ BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
+ ObjectAllocator>>>;
internal sealed unsafe partial class ClusterSession : IClusterSession
{
@@ -107,7 +110,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
continue;
}
- diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.mainStoreFunctions);
+ diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.storeFunctions);
var slot = HashSlotUtils.HashSlot(diskLogRecord.Key);
if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state
{
@@ -119,7 +122,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
// Set if key replace flag is set or key does not exist
var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key);
if (replaceOption || !Exists(keySlice))
- _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Main);
+ _ = basicGarnetApi.SET_Main(in diskLogRecord);
diskLogRecord.Dispose();
i++;
}
@@ -136,7 +139,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
var i = 0;
TrackImportProgress(keyCount, isMainStore: false, keyCount == 0);
var storeWrapper = clusterProvider.storeWrapper;
- var transientObjectIdMap = storeWrapper.objectStore.Log.TransientObjectIdMap;
+ var transientObjectIdMap = storeWrapper.store.Log.TransientObjectIdMap;
// Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord.
DiskLogRecord diskLogRecord = default;
@@ -151,7 +154,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
if (migrateState > 0)
continue;
- diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.objectStoreFunctions);
+ diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.storeFunctions);
var slot = HashSlotUtils.HashSlot(diskLogRecord.Key);
if (!currentConfig.IsImportingSlot(slot)) // Slot is not in importing state
{
@@ -162,7 +165,7 @@ void Process(BasicGarnetApi basicGarnetApi, byte[] input, string storeTypeSpan,
// Set if key replace flag is set or key does not exist
var keySlice = PinnedSpanByte.FromPinnedSpan(diskLogRecord.Key);
if (replaceOption || !Exists(keySlice))
- _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Object);
+ _ = basicGarnetApi.SET_Object(in diskLogRecord);
diskLogRecord.Dispose();
i++;
}
diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs
index eac6f680ce2..317e55f196a 100644
--- a/libs/cluster/Session/RespClusterReplicationCommands.cs
+++ b/libs/cluster/Session/RespClusterReplicationCommands.cs
@@ -348,26 +348,25 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters)
invalidParameters = false;
// Expecting exactly 7 arguments
- if (parseState.Count != 7)
+ if (parseState.Count != 6)
{
invalidParameters = true;
return true;
}
if (!parseState.TryGetBool(0, out var recoverMainStoreFromToken) ||
- !parseState.TryGetBool(1, out var recoverObjectStoreFromToken) ||
- !parseState.TryGetBool(2, out var replayAOF))
+ !parseState.TryGetBool(1, out var replayAOF))
{
while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_BOOLEAN, ref dcurr, dend))
SendAndReset();
return true;
}
- var primaryReplicaId = parseState.GetString(3);
- var checkpointEntryBytes = parseState.GetArgSliceByRef(4).ToArray();
+ var primaryReplicaId = parseState.GetString(2);
+ var checkpointEntryBytes = parseState.GetArgSliceByRef(3).ToArray();
- if (!parseState.TryGetLong(5, out var beginAddress) ||
- !parseState.TryGetLong(6, out var tailAddress))
+ if (!parseState.TryGetLong(4, out var beginAddress) ||
+ !parseState.TryGetLong(5, out var tailAddress))
{
while (!RespWriteUtils.TryWriteError(CmdStrings.RESP_ERR_GENERIC_VALUE_IS_NOT_INTEGER, ref dcurr, dend))
SendAndReset();
@@ -377,7 +376,6 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters)
var entry = CheckpointEntry.FromByteArray(checkpointEntryBytes);
var replicationOffset = clusterProvider.replicationManager.BeginReplicaRecover(
recoverMainStoreFromToken,
- recoverObjectStoreFromToken,
replayAOF,
primaryReplicaId,
entry,
@@ -479,8 +477,8 @@ private bool NetworkClusterSync(out bool invalidParameters)
if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr))
return false;
- diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.mainStoreFunctions);
- _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Main);
+ diskLogRecord = DiskLogRecord.Deserialize(recordSpan, valueObjectSerializer: default, transientObjectIdMap: default, storeWrapper.storeFunctions);
+ _ = basicGarnetApi.SET_Main(in diskLogRecord);
diskLogRecord.Dispose();
i++;
}
@@ -494,7 +492,7 @@ private bool NetworkClusterSync(out bool invalidParameters)
{
TrackImportProgress(recordCount, isMainStore: false, recordCount == 0);
var storeWrapper = clusterProvider.storeWrapper;
- var transientObjectIdMap = storeWrapper.objectStore.Log.TransientObjectIdMap;
+ var transientObjectIdMap = storeWrapper.store.Log.TransientObjectIdMap;
// Use try/finally instead of "using" because we don't want the boxing that an interface call would entail. Double-Dispose() is OK for DiskLogRecord.
DiskLogRecord diskLogRecord = default;
@@ -505,8 +503,8 @@ private bool NetworkClusterSync(out bool invalidParameters)
if (!RespReadUtils.GetSerializedRecordSpan(out var recordSpan, ref payloadPtr, payloadEndPtr))
return false;
- diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.objectStoreFunctions);
- _ = basicGarnetApi.SET(in diskLogRecord, StoreType.Object);
+ diskLogRecord = DiskLogRecord.Deserialize(recordSpan, storeWrapper.GarnetObjectSerializer, transientObjectIdMap, storeWrapper.storeFunctions);
+ _ = basicGarnetApi.SET_Object(in diskLogRecord);
diskLogRecord.Dispose();
i++;
}
diff --git a/libs/cluster/Session/RespClusterSlotManagementCommands.cs b/libs/cluster/Session/RespClusterSlotManagementCommands.cs
index 692f37ba8b9..a47b6509833 100644
--- a/libs/cluster/Session/RespClusterSlotManagementCommands.cs
+++ b/libs/cluster/Session/RespClusterSlotManagementCommands.cs
@@ -288,8 +288,7 @@ private bool NetworkClusterDelKeysInSlot(out bool invalidParameters)
var slots = new HashSet { slot };
ClusterManager.DeleteKeysInSlotsFromMainStore(basicGarnetApi, slots);
- if (!clusterProvider.serverOptions.DisableObjects)
- ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots);
+ ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots);
while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend))
SendAndReset();
@@ -325,8 +324,7 @@ private bool NetworkClusterDelKeysInSlotRange(out bool invalidParameters)
}
ClusterManager.DeleteKeysInSlotsFromMainStore(basicGarnetApi, slots);
- if (!clusterProvider.serverOptions.DisableObjects)
- ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots);
+ ClusterManager.DeleteKeysInSlotsFromObjectStore(basicGarnetApi, slots);
while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend))
SendAndReset();
diff --git a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
index bcb34cd55f1..4d5afeb0c18 100644
--- a/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
+++ b/libs/cluster/Session/SlotVerification/ClusterSlotVerify.cs
@@ -14,7 +14,7 @@ internal sealed unsafe partial class ClusterSession : IClusterSession
{
[MethodImpl(MethodImplOptions.AggressiveInlining)]
private bool Exists(PinnedSpanByte keySlice)
- => basicGarnetApi.EXISTS(keySlice, StoreType.All) == GarnetStatus.OK;
+ => basicGarnetApi.EXISTS(keySlice) == GarnetStatus.OK;
private ClusterSlotVerificationResult SingleKeySlotVerify(ref ClusterConfig config, ref PinnedSpanByte keySlice, bool readOnly, byte SessionAsking, int slot = -1)
{
diff --git a/libs/common/ExceptionInjectionType.cs b/libs/common/ExceptionInjectionType.cs
index dd122b0cbfd..8dc62305fd3 100644
--- a/libs/common/ExceptionInjectionType.cs
+++ b/libs/common/ExceptionInjectionType.cs
@@ -53,5 +53,13 @@ public enum ExceptionInjectionType
/// Delay response on receive checkpoint to trigger timeout
///
Replication_Timeout_On_Receive_Checkpoint,
+ ///
+ /// Replication InProgress during disk-based replica attach sync operation
+ ///
+ Replication_InProgress_During_DiskBased_Replica_Attach_Sync,
+ ///
+ /// Replication InProgress during diskless replica attach sync operation
+ ///
+ Replication_InProgress_During_Diskless_Replica_Attach_Sync,
}
}
\ No newline at end of file
diff --git a/libs/common/Metrics/InfoMetricsType.cs b/libs/common/Metrics/InfoMetricsType.cs
index e309b4cd9b9..2e68f6798a7 100644
--- a/libs/common/Metrics/InfoMetricsType.cs
+++ b/libs/common/Metrics/InfoMetricsType.cs
@@ -39,26 +39,14 @@ public enum InfoMetricsType : byte
///
STORE,
///
- /// Object store info
- ///
- OBJECTSTORE,
- ///
/// Store hash table info
///
STOREHASHTABLE,
///
- /// Object store hash table info
- ///
- OBJECTSTOREHASHTABLE,
- ///
/// Store revivification info
///
STOREREVIV,
///
- /// Object store hash table info
- ///
- OBJECTSTOREREVIV,
- ///
/// Persistence information
///
PERSISTENCE,
diff --git a/libs/common/RespMemoryWriter.cs b/libs/common/RespMemoryWriter.cs
index 8d6ee383374..ebb864855c1 100644
--- a/libs/common/RespMemoryWriter.cs
+++ b/libs/common/RespMemoryWriter.cs
@@ -378,6 +378,17 @@ public void WriteSimpleString(ReadOnlySpan simpleString)
ReallocateOutput(simpleString.Length);
}
+ ///
+ /// Write simple string to memory.
+ ///
+ /// An ASCII encoded simple string. The string mustn't contain a CR (\r) or LF (\n) bytes.
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ public void WriteSimpleString(ReadOnlySpan simpleString)
+ {
+ while (!RespWriteUtils.TryWriteSimpleString(simpleString, ref curr, end))
+ ReallocateOutput(simpleString.Length);
+ }
+
///
/// Write RESP3 true
///
diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs
index b7826d8f79c..d1db2b6fba2 100644
--- a/libs/host/Configuration/Options.cs
+++ b/libs/host/Configuration/Options.cs
@@ -90,48 +90,12 @@ internal sealed class Options : ICloneable
public string ReadCachePageSize { get; set; }
[MemorySizeValidation(false)]
- [Option("obj-heap-memory", Required = false, HelpText = "Object store heap memory size in bytes (Sum of size taken up by all object instances in the heap)")]
- public string ObjectStoreHeapMemorySize { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-log-memory", Required = false, HelpText = "Object store log memory used in bytes (Size of only the log with references to heap objects, excludes size of heap memory consumed by the objects themselves referred to from the log)")]
- public string ObjectStoreLogMemorySize { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-page", Required = false, HelpText = "Size of each object store page in bytes (rounds down to power of 2)")]
- public string ObjectStorePageSize { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-segment", Required = false, HelpText = "Size of each object store log segment in bytes on disk (rounds down to power of 2)")]
- public string ObjectStoreSegmentSize { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-index", Required = false, HelpText = "Start size of object store hash index in bytes (rounds down to power of 2)")]
- public string ObjectStoreIndexSize { get; set; }
-
- [MemorySizeValidation(false)]
- [Option("obj-index-max-size", Required = false, HelpText = "Max size of object store hash index in bytes (rounds down to power of 2)")]
- public string ObjectStoreIndexMaxSize { get; set; }
-
- [PercentageValidation]
- [Option("obj-mutable-percent", Required = false, HelpText = "Percentage of object store log memory that is kept mutable")]
- public int ObjectStoreMutablePercent { get; set; }
-
- [OptionValidation]
- [Option("obj-readcache", Required = false, HelpText = "Enables object store read cache for faster access to on-disk records.")]
- public bool? EnableObjectStoreReadCache { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-readcache-log-memory", Required = false, HelpText = "Total object store read cache log memory used in bytes (rounds down to power of 2)")]
- public string ObjectStoreReadCacheLogMemorySize { get; set; }
-
- [MemorySizeValidation]
- [Option("obj-readcache-page", Required = false, HelpText = "Size of each object store read cache page in bytes (rounds down to power of 2)")]
- public string ObjectStoreReadCachePageSize { get; set; }
+ [Option("heap-memory", Required = false, HelpText = "Heap memory size in bytes (Sum of size taken up by all object instances in the heap)")]
+ public string HeapMemorySize { get; set; }
[MemorySizeValidation(false)]
- [Option("obj-readcache-heap-memory", Required = false, HelpText = "Object store read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap)")]
- public string ObjectStoreReadCacheHeapMemorySize { get; set; }
+ [Option("readcache-heap-memory", Required = false, HelpText = "Read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap)")]
+ public string ReadCacheHeapMemorySize { get; set; }
[OptionValidation]
[Option("storage-tier", Required = false, HelpText = "Enable tiering of records (hybrid log) to storage, to support a larger-than-memory store. Use --logdir to specify storage directory.")]
@@ -141,10 +105,6 @@ internal sealed class Options : ICloneable
[Option("copy-reads-to-tail", Required = false, HelpText = "When records are read from the main store's in-memory immutable region or storage device, copy them to the tail of the log.")]
public bool? CopyReadsToTail { get; set; }
- [OptionValidation]
- [Option("obj-copy-reads-to-tail", Required = false, HelpText = "When records are read from the object store's in-memory immutable region or storage device, copy them to the tail of the log.")]
- public bool? ObjectStoreCopyReadsToTail { get; set; }
-
[LogDirValidation(false, false)]
[Option('l', "logdir", Required = false, HelpText = "Storage directory for tiered records (hybrid log), if storage tiering (--storage) is enabled. Uses current directory if unspecified.")]
public string LogDir { get; set; }
@@ -169,10 +129,6 @@ internal sealed class Options : ICloneable
[Option("pubsub-pagesize", Required = false, HelpText = "Page size of log used for pub/sub (rounds down to power of 2)")]
public string PubSubPageSize { get; set; }
- [OptionValidation]
- [Option("no-obj", Required = false, HelpText = "Disable support for data structure objects.")]
- public bool? DisableObjects { get; set; }
-
[OptionValidation]
[Option("cluster", Required = false, HelpText = "Enable cluster.")]
public bool? EnableCluster { get; set; }
@@ -273,10 +229,6 @@ internal sealed class Options : ICloneable
[Option("compaction-max-segments", Required = false, HelpText = "Number of log segments created on disk before compaction triggers.")]
public int CompactionMaxSegments { get; set; }
- [IntRangeValidation(0, int.MaxValue)]
- [Option("obj-compaction-max-segments", Required = false, HelpText = "Number of object store log segments created on disk before compaction triggers.")]
- public int ObjectStoreCompactionMaxSegments { get; set; }
-
[OptionValidation]
[Option("lua", Required = false, HelpText = "Enable Lua scripts on server.")]
public bool? EnableLua { get; set; }
@@ -518,8 +470,7 @@ internal sealed class Options : ICloneable
[DoubleRangeValidation(0, 1)]
[Option("reviv-fraction", Required = false,
- HelpText = "#: Fraction of mutable in-memory log space, from the highest log address down to the read-only region, that is eligible for revivification." +
- " Applies to both main and object store.")]
+ HelpText = "#: Fraction of mutable in-memory log space, from the highest log address down to the read-only region, that is eligible for revivification.")]
public double RevivifiableFraction { get; set; }
[OptionValidation]
@@ -545,15 +496,9 @@ internal sealed class Options : ICloneable
[OptionValidation]
[Option("reviv-in-chain-only", Required = false,
HelpText = "Revivify tombstoned records in tag chains only (do not use free list)." +
- " Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts. Propagates to object store by default.")]
+ " Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts.")]
public bool? RevivInChainOnly { get; set; }
- [IntRangeValidation(0, int.MaxValue)]
- [Option("reviv-obj-bin-record-count", Required = false,
- HelpText = "Number of records in the single free record bin for the object store. The Object store has only a single bin, unlike the main store." +
- " Ignored unless the main store is using the free record list.")]
- public int RevivObjBinRecordCount { get; set; }
-
[IntRangeValidation(0, int.MaxValue)]
[Option("object-scan-count-limit", Required = false, HelpText = "Limit of items to return in one iteration of *SCAN command")]
public int ObjectScanCountLimit { get; set; }
@@ -823,27 +768,16 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null)
EnableReadCache = EnableReadCache.GetValueOrDefault(),
ReadCacheMemorySize = ReadCacheMemorySize,
ReadCachePageSize = ReadCachePageSize,
- ObjectStoreHeapMemorySize = ObjectStoreHeapMemorySize,
- ObjectStoreLogMemorySize = ObjectStoreLogMemorySize,
- ObjectStorePageSize = ObjectStorePageSize,
- ObjectStoreSegmentSize = ObjectStoreSegmentSize,
- ObjectStoreIndexSize = ObjectStoreIndexSize,
- ObjectStoreIndexMaxSize = ObjectStoreIndexMaxSize,
- ObjectStoreMutablePercent = ObjectStoreMutablePercent,
- EnableObjectStoreReadCache = EnableObjectStoreReadCache.GetValueOrDefault(),
- ObjectStoreReadCachePageSize = ObjectStoreReadCachePageSize,
- ObjectStoreReadCacheLogMemorySize = ObjectStoreReadCacheLogMemorySize,
- ObjectStoreReadCacheHeapMemorySize = ObjectStoreReadCacheHeapMemorySize,
+ HeapMemorySize = HeapMemorySize,
+ ReadCacheHeapMemorySize = ReadCacheHeapMemorySize,
EnableStorageTier = enableStorageTier,
CopyReadsToTail = CopyReadsToTail.GetValueOrDefault(),
- ObjectStoreCopyReadsToTail = ObjectStoreCopyReadsToTail.GetValueOrDefault(),
LogDir = logDir,
CheckpointDir = checkpointDir,
Recover = Recover.GetValueOrDefault(),
EnableIncrementalSnapshots = EnableIncrementalSnapshots.GetValueOrDefault(),
DisablePubSub = DisablePubSub.GetValueOrDefault(),
PubSubPageSize = PubSubPageSize,
- DisableObjects = DisableObjects.GetValueOrDefault(),
EnableCluster = EnableCluster.GetValueOrDefault(),
CleanClusterConfig = CleanClusterConfig.GetValueOrDefault(),
ParallelMigrateTaskCount = ParallelMigrateTaskCount,
@@ -863,7 +797,6 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null)
CompactionType = CompactionType,
CompactionForceDelete = CompactionForceDelete.GetValueOrDefault(),
CompactionMaxSegments = CompactionMaxSegments,
- ObjectStoreCompactionMaxSegments = ObjectStoreCompactionMaxSegments,
GossipSamplePercent = GossipSamplePercent,
GossipDelay = GossipDelay,
ClusterTimeout = ClusterTimeout,
@@ -920,7 +853,6 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null)
RevivBinBestFitScanLimit = RevivBinBestFitScanLimit,
RevivNumberOfBinsToSearch = RevivNumberOfBinsToSearch,
RevivInChainOnly = RevivInChainOnly.GetValueOrDefault(),
- RevivObjBinRecordCount = RevivObjBinRecordCount,
EnableDebugCommand = EnableDebugCommand,
EnableModuleCommand = EnableModuleCommand,
ExtensionBinPaths = FileUtils.ConvertToAbsolutePaths(ExtensionBinPaths),
diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs
index f8cc704fb89..380f85dd744 100644
--- a/libs/host/GarnetServer.cs
+++ b/libs/host/GarnetServer.cs
@@ -20,11 +20,8 @@
namespace Garnet
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Implementation Garnet server
@@ -49,7 +46,6 @@ static string GetVersion()
private IGarnetServer[] servers;
private SubscribeBroker subscribeBroker;
private KVSettings kvSettings;
- private KVSettings objKvSettings;
private INamedDeviceFactory logFactory;
private MemoryLogger initLogger;
private ILogger logger;
@@ -264,13 +260,10 @@ private void InitializeServer()
var configMemoryLimit = (storeWrapper.store.IndexSize * 64) +
storeWrapper.store.Log.MaxMemorySizeBytes +
(storeWrapper.store.ReadCache?.MaxMemorySizeBytes ?? 0) +
- (storeWrapper.appendOnlyFile?.MaxMemorySizeBytes ?? 0);
- if (storeWrapper.objectStore != null)
- configMemoryLimit += (storeWrapper.objectStore.IndexSize * 64) +
- storeWrapper.objectStore.Log.MaxMemorySizeBytes +
- (storeWrapper.objectStore.ReadCache?.MaxMemorySizeBytes ?? 0) +
- (storeWrapper.objectStoreSizeTracker?.TargetSize ?? 0) +
- (storeWrapper.objectStoreSizeTracker?.ReadCacheTargetSize ?? 0);
+ (storeWrapper.appendOnlyFile?.MaxMemorySizeBytes ?? 0) +
+ (storeWrapper.sizeTracker?.TargetSize ?? 0) +
+ (storeWrapper.sizeTracker?.ReadCacheTargetSize ?? 0);
+
logger.LogInformation("Total configured memory limit: {configMemoryLimit}", configMemoryLimit);
}
@@ -300,12 +293,11 @@ private void InitializeServer()
private GarnetDatabase CreateDatabase(int dbId, GarnetServerOptions serverOptions, ClusterFactory clusterFactory,
CustomCommandManager customCommandManager)
{
- var store = CreateMainStore(dbId, clusterFactory, out var epoch, out var stateMachineDriver);
- var objectStore = CreateObjectStore(dbId, clusterFactory, customCommandManager, epoch, stateMachineDriver, out var objectStoreSizeTracker);
+ var store = CreateStore(dbId, clusterFactory, customCommandManager, out var epoch, out var stateMachineDriver, out var sizeTracker);
var (aofDevice, aof) = CreateAOF(dbId);
- return new GarnetDatabase(dbId, store, objectStore, epoch, stateMachineDriver, objectStoreSizeTracker,
- aofDevice, aof, serverOptions.AdjustedIndexMaxCacheLines == 0,
- serverOptions.AdjustedObjectStoreIndexMaxCacheLines == 0);
+
+ return new GarnetDatabase(dbId, store, epoch, stateMachineDriver, sizeTracker,
+ aofDevice, aof, serverOptions.AdjustedIndexMaxCacheLines == 0);
}
private void LoadModules(CustomCommandManager customCommandManager)
@@ -331,60 +323,36 @@ private void LoadModules(CustomCommandManager customCommandManager)
}
}
- private TsavoriteKV CreateMainStore(int dbId, IClusterFactory clusterFactory,
- out LightEpoch epoch, out StateMachineDriver stateMachineDriver)
+ private TsavoriteKV CreateStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager,
+ out LightEpoch epoch, out StateMachineDriver stateMachineDriver, out CacheSizeTracker sizeTracker)
{
+ sizeTracker = null;
+
epoch = new LightEpoch();
stateMachineDriver = new StateMachineDriver(epoch, loggerFactory?.CreateLogger($"StateMachineDriver"));
- kvSettings = opts.GetSettings(loggerFactory, epoch, stateMachineDriver, out logFactory);
+ kvSettings = opts.GetSettings(loggerFactory, epoch, stateMachineDriver, out logFactory, out var heapMemorySize, out var readCacheHeapMemorySize);
// Run checkpoint on its own thread to control p99
kvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs;
- var baseName = opts.GetMainStoreCheckpointDirectory(dbId);
+ var baseName = opts.GetStoreCheckpointDirectory(dbId);
var defaultNamingScheme = new DefaultCheckpointNamingScheme(baseName);
kvSettings.CheckpointManager = opts.EnableCluster ?
clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: true, logger) :
new GarnetCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true);
- return new(kvSettings
- , StoreFunctions.Create()
- , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions));
- }
-
- private TsavoriteKV CreateObjectStore(int dbId, IClusterFactory clusterFactory, CustomCommandManager customCommandManager,
- LightEpoch epoch, StateMachineDriver stateMachineDriver, out CacheSizeTracker objectStoreSizeTracker)
- {
- objectStoreSizeTracker = null;
- if (opts.DisableObjects)
- return null;
-
- objKvSettings = opts.GetObjectStoreSettings(loggerFactory, epoch, stateMachineDriver,
- out var objHeapMemorySize, out var objReadCacheHeapMemorySize);
-
- // Run checkpoint on its own thread to control p99
- objKvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs;
-
- var baseName = opts.GetObjectStoreCheckpointDirectory(dbId);
- var defaultNamingScheme = new DefaultCheckpointNamingScheme(baseName);
-
- objKvSettings.CheckpointManager = opts.EnableCluster ?
- clusterFactory.CreateCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, isMainStore: false, logger) :
- new GarnetCheckpointManager(opts.DeviceFactoryCreator, defaultNamingScheme, removeOutdated: true);
-
- var objStore = new TsavoriteKV(objKvSettings
- , StoreFunctions.Create(new SpanByteComparer(),
+ var store = new TsavoriteKV(kvSettings
+ , Tsavorite.core.StoreFunctions.Create(new SpanByteComparer(),
() => new GarnetObjectSerializer(customCommandManager))
, (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions));
- if (objHeapMemorySize > 0 || objReadCacheHeapMemorySize > 0)
- objectStoreSizeTracker = new CacheSizeTracker(objStore, objKvSettings, objHeapMemorySize, objReadCacheHeapMemorySize,
+ if (heapMemorySize > 0 || readCacheHeapMemorySize > 0)
+ sizeTracker = new CacheSizeTracker(store, kvSettings, heapMemorySize, readCacheHeapMemorySize,
this.loggerFactory);
- return objStore;
-
+ return store;
}
private (IDevice, TsavoriteLog) CreateAOF(int dbId)
@@ -454,11 +422,6 @@ private void InternalDispose()
servers[i]?.Dispose();
subscribeBroker?.Dispose();
kvSettings.LogDevice?.Dispose();
- if (!opts.DisableObjects)
- {
- objKvSettings.LogDevice?.Dispose();
- objKvSettings.ObjectLogDevice?.Dispose();
- }
opts.AuthSettings?.Dispose();
if (disposeLoggerFactory)
loggerFactory?.Dispose();
diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf
index af7be040685..a2ea1cc575d 100644
--- a/libs/host/defaults.conf
+++ b/libs/host/defaults.conf
@@ -42,48 +42,18 @@
/* Size of each read cache page in bytes (rounds down to power of 2) */
"ReadCachePageSize" : "32m",
- /* Object store heap memory size in bytes (Sum of size taken up by all object instances in the heap) */
- "ObjectStoreHeapMemorySize" : "",
+ /* Heap memory size in bytes (Sum of size taken up by all object instances in the heap) */
+ "HeapMemorySize" : "",
- /* Object store log memory used in bytes (Size of only the log with references to heap objects, excludes size of heap memory consumed by the objects themselves referred to from the log) */
- "ObjectStoreLogMemorySize" : "32m",
-
- /* Size of each object store page in bytes (rounds down to power of 2) */
- "ObjectStorePageSize" : "4k",
-
- /* Size of each object store log segment in bytes on disk (rounds down to power of 2) */
- "ObjectStoreSegmentSize" : "32m",
-
- /* Start size of object store hash index in bytes (rounds down to power of 2) */
- "ObjectStoreIndexSize" : "16m",
-
- /* Max size of object store hash index in bytes (rounds down to power of 2) */
- "ObjectStoreIndexMaxSize": "",
-
- /* Percentage of object store log memory that is kept mutable */
- "ObjectStoreMutablePercent" : 90,
-
- /* Enables object store read cache for faster access to on-disk records */
- "EnableObjectStoreReadCache" : false,
-
- /* Total object store read cache log memory used in bytes (rounds down to power of 2) */
- "ObjectStoreReadCacheLogMemorySize" : "32m",
-
- /* Size of each object store read cache page in bytes (rounds down to power of 2) */
- "ObjectStoreReadCachePageSize" : "1m",
-
- /* Object store read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap) */
- "ObjectStoreReadCacheHeapMemorySize" : "",
+ /* Read cache heap memory size in bytes (Sum of size taken up by all object instances in the heap) */
+ "ReadCacheHeapMemorySize" : "",
/* Enable tiering of records (hybrid log) to storage, to support a larger-than-memory store. Use --logdir to specify storage directory. */
"EnableStorageTier" : false,
- /* When records are read from the main store's in-memory immutable region or storage device, copy them to the tail of the log. */
+ /* When records are read from the main store''s in-memory immutable region or storage device, copy them to the tail of the log. */
"CopyReadsToTail" : false,
- /* When records are read from the object store's in-memory immutable region or storage device, copy them to the tail of the log. */
- "ObjectStoreCopyReadsToTail" : false,
-
/* Storage directory for tiered records (hybrid log), if storage tiering (--storage) is enabled. Uses current directory if unspecified. */
"LogDir" : null,
@@ -102,9 +72,6 @@
/* Page size of log used for pub/sub (rounds down to power of 2) */
"PubSubPageSize" : "4k",
- /* Disable support for data structure objects. */
- "DisableObjects" : false,
-
/* Enable cluster. */
"EnableCluster" : false,
@@ -191,9 +158,6 @@
/* Number of log segments created on disk before compaction triggers. */
"CompactionMaxSegments" : 32,
- /* Number of object store log segments created on disk before compaction triggers. */
- "ObjectStoreCompactionMaxSegments" : 32,
-
/* Enable Lua scripts on server. */
"EnableLua" : false,
@@ -368,9 +332,6 @@
/* Revivify tombstoned records in tag chains only (do not use free list). Cannot be used with reviv-bin-record-sizes or reviv-bin-record-counts. Propagates to object store by default. */
"RevivInChainOnly" : false,
- /* Number of records in the single free record bin for the object store. The Object store has only a single bin, unlike the main store. Ignored unless the main store is using the free record list. */
- "RevivObjBinRecordCount" : 256,
-
/* Limit of items to return in one iteration of *SCAN command */
"ObjectScanCountLimit" : 1000,
diff --git a/libs/resources/RespCommandsDocs.json b/libs/resources/RespCommandsDocs.json
index e4d86c25bb6..be77703a3ed 100644
--- a/libs/resources/RespCommandsDocs.json
+++ b/libs/resources/RespCommandsDocs.json
@@ -41,13 +41,14 @@
{
"Command": "ACL_GENPASS",
"Name": "ACL|GENPASS",
- "Summary": "Generate a pseudorandom secure password to use for ACL users",
+ "Summary": "Generates a pseudorandom, secure password that can be used to identify ACL users.",
"Group": "Server",
"Complexity": "O(1)",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "BITS",
+ "DisplayText": "bits",
"Type": "Integer",
"ArgumentFlags": "Optional"
}
@@ -127,21 +128,6 @@
}
]
},
- {
- "Command": "EXPDELSCAN",
- "Name": "EXPDELSCAN",
- "Summary": "Scans mutable records to delete expired ones.",
- "Group": "Generic",
- "Complexity": "O(1) for every call. O(N) for a complete iteration. N is the number of records in memory.",
- "Arguments": [
- {
- "TypeDiscriminator": "RespCommandBasicArgument",
- "Name": "DBID",
- "DisplayText": "dbid",
- "Type": "Integer"
- }
- ]
- },
{
"Command": "APPEND",
"Name": "APPEND",
@@ -1383,6 +1369,13 @@
}
]
},
+ {
+ "Command": "CLUSTER_FLUSHALL",
+ "Name": "CLUSTER|FLUSHALL",
+ "Summary": "Sent by primary to replica to force to FLUSH its database",
+ "Group": "Cluster",
+ "Complexity": "O(1)"
+ },
{
"Command": "CLUSTER_FORGET",
"Name": "CLUSTER|FORGET",
@@ -1925,22 +1918,6 @@
}
]
},
- {
- "Command": "DELIFEXPIM",
- "Name": "DELIFEXPIM",
- "Summary": "Deletes a key if it is in memory and expired.",
- "Group": "Generic",
- "Complexity": "O(1)",
- "Arguments": [
- {
- "TypeDiscriminator": "RespCommandKeyArgument",
- "Name": "KEY",
- "DisplayText": "key",
- "Type": "Key",
- "KeySpecIndex": 0
- }
- ]
- },
{
"Command": "DELIFGREATER",
"Name": "DELIFGREATER",
@@ -2097,6 +2074,21 @@
}
]
},
+ {
+ "Command": "EXPDELSCAN",
+ "Name": "EXPDELSCAN",
+ "Summary": "Scans mutable records to delete expired ones.",
+ "Group": "Generic",
+ "Complexity": "O(1) for every call. O(N) for a complete iteration. N is the number of records in memory.",
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandBasicArgument",
+ "Name": "DBID",
+ "DisplayText": "dbid",
+ "Type": "Integer"
+ }
+ ]
+ },
{
"Command": "EXPIRE",
"Name": "EXPIRE",
@@ -2536,7 +2528,7 @@
{
"Command": "GEORADIUS",
"Name": "GEORADIUS",
- "Summary": "Query a sorted set representing a geospatial index to fetch members matching a given maximum distance from a point",
+ "Summary": "Queries a geospatial index for members within a distance from a coordinate, optionally stores the result.",
"Group": "Geo",
"Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.",
"DocFlags": "Deprecated",
@@ -2545,22 +2537,26 @@
{
"TypeDiscriminator": "RespCommandKeyArgument",
"Name": "KEY",
+ "DisplayText": "key",
"Type": "Key",
"KeySpecIndex": 0
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "LONGITUDE",
+ "DisplayText": "longitude",
"Type": "Double"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "LATITUDE",
+ "DisplayText": "latitude",
"Type": "Double"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "RADIUS",
+ "DisplayText": "radius",
"Type": "Double"
},
{
@@ -2571,24 +2567,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "M",
+ "DisplayText": "m",
"Type": "PureToken",
"Token": "M"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "KM",
+ "DisplayText": "km",
"Type": "PureToken",
"Token": "KM"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "FT",
+ "DisplayText": "ft",
"Type": "PureToken",
"Token": "FT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MI",
+ "DisplayText": "mi",
"Type": "PureToken",
"Token": "MI"
}
@@ -2597,6 +2597,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHCOORD",
+ "DisplayText": "withcoord",
"Type": "PureToken",
"Token": "WITHCOORD",
"ArgumentFlags": "Optional"
@@ -2604,6 +2605,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHDIST",
+ "DisplayText": "withdist",
"Type": "PureToken",
"Token": "WITHDIST",
"ArgumentFlags": "Optional"
@@ -2611,25 +2613,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHHASH",
+ "DisplayText": "withhash",
"Type": "PureToken",
"Token": "WITHHASH",
"ArgumentFlags": "Optional"
},
{
"TypeDiscriminator": "RespCommandContainerArgument",
- "Name": "COUNT",
+ "Name": "COUNT-BLOCK",
"Type": "Block",
"ArgumentFlags": "Optional",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "COUNT",
+ "DisplayText": "count",
"Type": "Integer",
"Token": "COUNT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ANY",
+ "DisplayText": "any",
"Type": "PureToken",
"Token": "ANY",
"ArgumentFlags": "Optional"
@@ -2645,39 +2650,49 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ASC",
+ "DisplayText": "asc",
"Type": "PureToken",
"Token": "ASC"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "DESC",
+ "DisplayText": "desc",
"Type": "PureToken",
"Token": "DESC"
}
]
},
{
- "TypeDiscriminator": "RespCommandKeyArgument",
- "Name": "KEY",
- "Type": "Key",
- "Token": "STORE",
- "ArgumentFlags": "Optional",
- "KeySpecIndex": 1
- },
- {
- "TypeDiscriminator": "RespCommandKeyArgument",
- "Name": "KEY",
- "Type": "Key",
- "Token": "STOREDIST",
+ "TypeDiscriminator": "RespCommandContainerArgument",
+ "Name": "STORE",
+ "Type": "OneOf",
"ArgumentFlags": "Optional",
- "KeySpecIndex": 2
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "STOREKEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "Token": "STORE",
+ "KeySpecIndex": 1
+ },
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "STOREDISTKEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "Token": "STOREDIST",
+ "KeySpecIndex": 2
+ }
+ ]
}
]
},
{
"Command": "GEORADIUS_RO",
"Name": "GEORADIUS_RO",
- "Summary": "A read-only variant for GEORADIUS",
+ "Summary": "Returns members from a geospatial index that are within a distance from a coordinate.",
"Group": "Geo",
"Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.",
"DocFlags": "Deprecated",
@@ -2686,22 +2701,26 @@
{
"TypeDiscriminator": "RespCommandKeyArgument",
"Name": "KEY",
+ "DisplayText": "key",
"Type": "Key",
"KeySpecIndex": 0
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "LONGITUDE",
+ "DisplayText": "longitude",
"Type": "Double"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "LATITUDE",
+ "DisplayText": "latitude",
"Type": "Double"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "RADIUS",
+ "DisplayText": "radius",
"Type": "Double"
},
{
@@ -2712,24 +2731,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "M",
+ "DisplayText": "m",
"Type": "PureToken",
"Token": "M"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "KM",
+ "DisplayText": "km",
"Type": "PureToken",
"Token": "KM"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "FT",
+ "DisplayText": "ft",
"Type": "PureToken",
"Token": "FT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MI",
+ "DisplayText": "mi",
"Type": "PureToken",
"Token": "MI"
}
@@ -2738,6 +2761,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHCOORD",
+ "DisplayText": "withcoord",
"Type": "PureToken",
"Token": "WITHCOORD",
"ArgumentFlags": "Optional"
@@ -2745,6 +2769,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHDIST",
+ "DisplayText": "withdist",
"Type": "PureToken",
"Token": "WITHDIST",
"ArgumentFlags": "Optional"
@@ -2752,25 +2777,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHHASH",
+ "DisplayText": "withhash",
"Type": "PureToken",
"Token": "WITHHASH",
"ArgumentFlags": "Optional"
},
{
"TypeDiscriminator": "RespCommandContainerArgument",
- "Name": "COUNT",
+ "Name": "COUNT-BLOCK",
"Type": "Block",
"ArgumentFlags": "Optional",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "COUNT",
+ "DisplayText": "count",
"Type": "Integer",
"Token": "COUNT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ANY",
+ "DisplayText": "any",
"Type": "PureToken",
"Token": "ANY",
"ArgumentFlags": "Optional"
@@ -2786,12 +2814,14 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ASC",
+ "DisplayText": "asc",
"Type": "PureToken",
"Token": "ASC"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "DESC",
+ "DisplayText": "desc",
"Type": "PureToken",
"Token": "DESC"
}
@@ -2802,7 +2832,7 @@
{
"Command": "GEORADIUSBYMEMBER",
"Name": "GEORADIUSBYMEMBER",
- "Summary": "Query a sorted set representing a geospatial index to fetch members matching a given maximum distance from a member",
+ "Summary": "Queries a geospatial index for members within a distance from a member, optionally stores the result.",
"Group": "Geo",
"Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.",
"DocFlags": "Deprecated",
@@ -2811,17 +2841,20 @@
{
"TypeDiscriminator": "RespCommandKeyArgument",
"Name": "KEY",
+ "DisplayText": "key",
"Type": "Key",
"KeySpecIndex": 0
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MEMBER",
+ "DisplayText": "member",
"Type": "String"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "RADIUS",
+ "DisplayText": "radius",
"Type": "Double"
},
{
@@ -2832,24 +2865,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "M",
+ "DisplayText": "m",
"Type": "PureToken",
"Token": "M"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "KM",
+ "DisplayText": "km",
"Type": "PureToken",
"Token": "KM"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "FT",
+ "DisplayText": "ft",
"Type": "PureToken",
"Token": "FT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MI",
+ "DisplayText": "mi",
"Type": "PureToken",
"Token": "MI"
}
@@ -2858,6 +2895,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHCOORD",
+ "DisplayText": "withcoord",
"Type": "PureToken",
"Token": "WITHCOORD",
"ArgumentFlags": "Optional"
@@ -2865,6 +2903,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHDIST",
+ "DisplayText": "withdist",
"Type": "PureToken",
"Token": "WITHDIST",
"ArgumentFlags": "Optional"
@@ -2872,25 +2911,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHHASH",
+ "DisplayText": "withhash",
"Type": "PureToken",
"Token": "WITHHASH",
"ArgumentFlags": "Optional"
},
{
"TypeDiscriminator": "RespCommandContainerArgument",
- "Name": "COUNT",
+ "Name": "COUNT-BLOCK",
"Type": "Block",
"ArgumentFlags": "Optional",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "COUNT",
+ "DisplayText": "count",
"Type": "Integer",
"Token": "COUNT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ANY",
+ "DisplayText": "any",
"Type": "PureToken",
"Token": "ANY",
"ArgumentFlags": "Optional"
@@ -2906,39 +2948,49 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ASC",
+ "DisplayText": "asc",
"Type": "PureToken",
"Token": "ASC"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "DESC",
+ "DisplayText": "desc",
"Type": "PureToken",
"Token": "DESC"
}
]
},
{
- "TypeDiscriminator": "RespCommandKeyArgument",
- "Name": "KEY",
- "Type": "Key",
- "Token": "STORE",
- "ArgumentFlags": "Optional",
- "KeySpecIndex": 1
- },
- {
- "TypeDiscriminator": "RespCommandKeyArgument",
- "Name": "KEY",
- "Type": "Key",
- "Token": "STOREDIST",
+ "TypeDiscriminator": "RespCommandContainerArgument",
+ "Name": "STORE",
+ "Type": "OneOf",
"ArgumentFlags": "Optional",
- "KeySpecIndex": 2
+ "Arguments": [
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "STOREKEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "Token": "STORE",
+ "KeySpecIndex": 1
+ },
+ {
+ "TypeDiscriminator": "RespCommandKeyArgument",
+ "Name": "STOREDISTKEY",
+ "DisplayText": "key",
+ "Type": "Key",
+ "Token": "STOREDIST",
+ "KeySpecIndex": 2
+ }
+ ]
}
]
},
{
"Command": "GEORADIUSBYMEMBER_RO",
"Name": "GEORADIUSBYMEMBER_RO",
- "Summary": "A read-only variant for GEORADIUSBYMEMBER",
+ "Summary": "Returns members from a geospatial index that are within a distance from a member.",
"Group": "Geo",
"Complexity": "O(N\u002Blog(M)) where N is the number of elements inside the bounding box of the circular area delimited by center and radius and M is the number of items inside the index.",
"DocFlags": "Deprecated",
@@ -2947,17 +2999,20 @@
{
"TypeDiscriminator": "RespCommandKeyArgument",
"Name": "KEY",
+ "DisplayText": "key",
"Type": "Key",
"KeySpecIndex": 0
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MEMBER",
+ "DisplayText": "member",
"Type": "String"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "RADIUS",
+ "DisplayText": "radius",
"Type": "Double"
},
{
@@ -2968,24 +3023,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "M",
+ "DisplayText": "m",
"Type": "PureToken",
"Token": "M"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "KM",
+ "DisplayText": "km",
"Type": "PureToken",
"Token": "KM"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "FT",
+ "DisplayText": "ft",
"Type": "PureToken",
"Token": "FT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "MI",
+ "DisplayText": "mi",
"Type": "PureToken",
"Token": "MI"
}
@@ -2994,6 +3053,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHCOORD",
+ "DisplayText": "withcoord",
"Type": "PureToken",
"Token": "WITHCOORD",
"ArgumentFlags": "Optional"
@@ -3001,6 +3061,7 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHDIST",
+ "DisplayText": "withdist",
"Type": "PureToken",
"Token": "WITHDIST",
"ArgumentFlags": "Optional"
@@ -3008,25 +3069,28 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "WITHHASH",
+ "DisplayText": "withhash",
"Type": "PureToken",
"Token": "WITHHASH",
"ArgumentFlags": "Optional"
},
{
"TypeDiscriminator": "RespCommandContainerArgument",
- "Name": "COUNT",
+ "Name": "COUNT-BLOCK",
"Type": "Block",
"ArgumentFlags": "Optional",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "COUNT",
+ "DisplayText": "count",
"Type": "Integer",
"Token": "COUNT"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ANY",
+ "DisplayText": "any",
"Type": "PureToken",
"Token": "ANY",
"ArgumentFlags": "Optional"
@@ -3042,12 +3106,14 @@
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "ASC",
+ "DisplayText": "asc",
"Type": "PureToken",
"Token": "ASC"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "DESC",
+ "DisplayText": "desc",
"Type": "PureToken",
"Token": "DESC"
}
@@ -4536,14 +4602,6 @@
"Type": "Integer",
"Token": "COUNT",
"ArgumentFlags": "Optional"
- },
- {
- "TypeDiscriminator": "RespCommandBasicArgument",
- "Name": "NOVALUES",
- "DisplayText": "novalues",
- "Type": "PureToken",
- "Token": "NOVALUES",
- "ArgumentFlags": "Optional"
}
]
},
@@ -6933,7 +6991,8 @@
"Name": "NOGET",
"DisplayText": "noget",
"Type": "PureToken",
- "Token": "NOGET"
+ "Token": "NOGET",
+ "ArgumentFlags": "Optional"
}
]
},
@@ -6990,7 +7049,8 @@
"Name": "NOGET",
"DisplayText": "noget",
"Type": "PureToken",
- "Token": "NOGET"
+ "Token": "NOGET",
+ "ArgumentFlags": "Optional"
}
]
},
@@ -7562,18 +7622,20 @@
{
"Command": "SWAPDB",
"Name": "SWAPDB",
- "Summary": "Swaps two Memurai databases",
+ "Summary": "Swaps two Redis databases.",
"Group": "Server",
"Complexity": "O(N) where N is the count of clients watching or blocking on keys from both databases.",
"Arguments": [
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "INDEX1",
+ "DisplayText": "index1",
"Type": "Integer"
},
{
"TypeDiscriminator": "RespCommandBasicArgument",
"Name": "INDEX2",
+ "DisplayText": "index2",
"Type": "Integer"
}
]
diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json
index f381ba1548f..daa1acd29d3 100644
--- a/libs/resources/RespCommandsInfo.json
+++ b/libs/resources/RespCommandsInfo.json
@@ -23,6 +23,20 @@
"response_policy:all_succeeded"
]
},
+ {
+ "Command": "ACL_GENPASS",
+ "Name": "ACL|GENPASS",
+ "Arity": -2,
+ "Flags": "Loading, NoScript, Stale",
+ "AclCategories": "Slow"
+ },
+ {
+ "Command": "ACL_GETUSER",
+ "Name": "ACL|GETUSER",
+ "Arity": 3,
+ "Flags": "Admin, Loading, NoScript, Stale",
+ "AclCategories": "Admin, Dangerous, Slow"
+ },
{
"Command": "ACL_LIST",
"Name": "ACL|LIST",
@@ -72,33 +86,9 @@
"Arity": 2,
"Flags": "Loading, NoScript, Stale",
"AclCategories": "Slow"
- },
- {
- "Command": "ACL_GETUSER",
- "Name": "ACL|GETUSER",
- "Arity": 3,
- "Flags": "Admin, Loading, NoScript, Stale",
- "AclCategories": "Admin, Dangerous, Slow"
- },
- {
- "Command": "ACL_GENPASS",
- "Name": "ACL|GENPASS",
- "Arity": -2,
- "Flags": "Loading, NoScript, Stale",
- "AclCategories": "Slow"
}
]
},
- {
- "Command": "EXPDELSCAN",
- "Name": "EXPDELSCAN",
- "Arity": -1,
- "Flags": "Admin, NoMulti, NoScript, ReadOnly",
- "FirstKey": 1,
- "LastKey": 1,
- "Step": 1,
- "AclCategories": "Admin, Garnet"
- },
{
"Command": "APPEND",
"Name": "APPEND",
@@ -122,7 +112,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "ASKING",
@@ -175,7 +166,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "BITFIELD",
@@ -201,7 +193,8 @@
"Notes": "This command allows both access and modification of the key",
"Flags": "RW, Access, Update, VariableFlags"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "BITFIELD_RO",
@@ -226,7 +219,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "BITOP",
@@ -264,7 +258,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "BITPOS",
@@ -289,7 +284,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "BLMOVE",
@@ -327,7 +323,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BLMPOP",
@@ -349,7 +346,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BLPOP",
@@ -374,7 +372,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BRPOP",
@@ -399,7 +398,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BRPOPLPUSH",
@@ -437,7 +437,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BZMPOP",
@@ -459,7 +460,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BZPOPMAX",
@@ -484,7 +486,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "BZPOPMIN",
@@ -509,7 +512,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "CLIENT",
@@ -727,6 +731,14 @@
"Flags": "Admin, NoMulti, NoScript",
"AclCategories": "Admin, Dangerous, Slow, Garnet"
},
+ {
+ "Command": "CLUSTER_FLUSHALL",
+ "Name": "CLUSTER|FLUSHALL",
+ "IsInternal": true,
+ "Arity": 2,
+ "Flags": "Admin, NoMulti, NoScript",
+ "AclCategories": "Admin, Dangerous, Slow, Garnet"
+ },
{
"Command": "CLUSTER_FORGET",
"Name": "CLUSTER|FORGET",
@@ -966,14 +978,6 @@
"Arity": 4,
"Flags": "Admin, NoMulti, NoScript",
"AclCategories": "Admin, Dangerous, Slow, Garnet"
- },
- {
- "Command": "CLUSTER_FLUSHALL",
- "Name": "CLUSTER|FLUSHALL",
- "IsInternal": true,
- "Arity": 2,
- "Flags": "Admin, NoMulti, NoScript",
- "AclCategories": "Admin, Dangerous, Slow, Garnet"
}
]
},
@@ -1085,14 +1089,16 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "Read, Slow, Garnet"
+ "AclCategories": "Read, Slow, Garnet",
+ "StoreType": "Object"
},
{
"Command": "CustomObjCmd",
"Name": "CustomObjCmd",
"IsInternal": true,
"Arity": 1,
- "AclCategories": "Dangerous, Slow, Garnet, Custom"
+ "AclCategories": "Dangerous, Slow, Garnet, Custom",
+ "StoreType": "Object"
},
{
"Command": "CustomProcedure",
@@ -1106,7 +1112,8 @@
"Name": "CustomRawStringCmd",
"IsInternal": true,
"Arity": 1,
- "AclCategories": "Dangerous, Slow, Garnet, Custom"
+ "AclCategories": "Dangerous, Slow, Garnet, Custom",
+ "StoreType": "Main"
},
{
"Command": "CustomTxn",
@@ -1156,7 +1163,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "DECRBY",
@@ -1181,7 +1189,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "DEL",
@@ -1210,7 +1219,8 @@
},
"Flags": "RM, Delete"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "DELIFGREATER",
@@ -1234,7 +1244,8 @@
},
"Flags": "RM, Delete"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "DISCARD",
@@ -1269,7 +1280,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "ECHO",
@@ -1357,7 +1369,18 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "All"
+ },
+ {
+ "Command": "EXPDELSCAN",
+ "Name": "EXPDELSCAN",
+ "Arity": -1,
+ "Flags": "Admin, NoMulti, NoScript, ReadOnly",
+ "FirstKey": 1,
+ "LastKey": 1,
+ "Step": 1,
+ "AclCategories": "Admin, Garnet"
},
{
"Command": "EXPIRE",
@@ -1382,7 +1405,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "EXPIREAT",
@@ -1407,7 +1431,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "EXPIRETIME",
@@ -1432,7 +1457,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "FAILOVER",
@@ -1496,7 +1522,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEODIST",
@@ -1521,7 +1548,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEOHASH",
@@ -1546,7 +1574,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEOPOS",
@@ -1571,7 +1600,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEORADIUS",
@@ -1624,7 +1654,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEORADIUS_RO",
@@ -1649,7 +1680,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEORADIUSBYMEMBER",
@@ -1702,7 +1734,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEORADIUSBYMEMBER_RO",
@@ -1727,7 +1760,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEOSEARCH",
@@ -1752,7 +1786,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GEOSEARCHSTORE",
@@ -1790,7 +1825,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "GET",
@@ -1815,7 +1851,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETBIT",
@@ -1840,7 +1877,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETDEL",
@@ -1865,7 +1903,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETEX",
@@ -1891,7 +1930,8 @@
"Notes": "RW and UPDATE because it changes the TTL",
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETIFNOTMATCH",
@@ -1915,7 +1955,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETRANGE",
@@ -1940,7 +1981,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETSET",
@@ -1965,7 +2007,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "GETWITHETAG",
@@ -1989,7 +2032,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "HCOLLECT",
@@ -2014,7 +2058,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HDEL",
@@ -2039,7 +2084,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HELLO",
@@ -2071,7 +2117,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HEXPIRE",
@@ -2096,7 +2143,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HEXPIREAT",
@@ -2121,7 +2169,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HEXPIRETIME",
@@ -2146,7 +2195,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HGET",
@@ -2171,7 +2221,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HGETALL",
@@ -2199,7 +2250,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HINCRBY",
@@ -2224,7 +2276,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HINCRBYFLOAT",
@@ -2249,7 +2302,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HKEYS",
@@ -2277,7 +2331,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HLEN",
@@ -2302,7 +2357,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HMGET",
@@ -2327,7 +2383,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HMSET",
@@ -2352,7 +2409,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HPERSIST",
@@ -2377,7 +2435,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HPEXPIRE",
@@ -2402,7 +2461,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HPEXPIREAT",
@@ -2427,7 +2487,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HPEXPIRETIME",
@@ -2452,7 +2513,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HPTTL",
@@ -2477,7 +2539,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HRANDFIELD",
@@ -2505,7 +2568,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HSCAN",
@@ -2533,7 +2597,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HSET",
@@ -2558,7 +2623,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HSETNX",
@@ -2583,7 +2649,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HSTRLEN",
@@ -2608,7 +2675,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HTTL",
@@ -2633,7 +2701,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "HVALS",
@@ -2661,7 +2730,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "INCR",
@@ -2686,7 +2756,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "INCRBY",
@@ -2711,7 +2782,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "INCRBYFLOAT",
@@ -2736,7 +2808,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "INFO",
@@ -2832,7 +2905,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "LINDEX",
@@ -2857,7 +2931,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LINSERT",
@@ -2882,7 +2957,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LLEN",
@@ -2907,7 +2983,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LMOVE",
@@ -2945,7 +3022,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LMPOP",
@@ -2967,7 +3045,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LPOP",
@@ -2992,7 +3071,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LPOS",
@@ -3017,7 +3097,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LPUSH",
@@ -3042,7 +3123,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LPUSHX",
@@ -3067,7 +3149,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LRANGE",
@@ -3092,7 +3175,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LREM",
@@ -3117,7 +3201,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LSET",
@@ -3142,7 +3227,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "LTRIM",
@@ -3167,7 +3253,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "MEMORY",
@@ -3228,7 +3315,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "MIGRATE",
@@ -3322,7 +3410,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "MSETNX",
@@ -3347,7 +3436,8 @@
},
"Flags": "OW, Insert"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "MULTI",
@@ -3379,7 +3469,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "PEXPIRE",
@@ -3404,7 +3495,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "PEXPIREAT",
@@ -3429,7 +3521,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "PEXPIRETIME",
@@ -3454,7 +3547,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "PFADD",
@@ -3479,7 +3573,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "PFCOUNT",
@@ -3505,7 +3600,8 @@
"Notes": "RW because it may change the internal representation of the key, and propagate to replicas",
"Flags": "RW, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "PFMERGE",
@@ -3543,7 +3639,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "PING",
@@ -3579,7 +3676,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "PSUBSCRIBE",
@@ -3614,7 +3712,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "PUBLISH",
@@ -3736,7 +3835,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "RENAMENX",
@@ -3774,7 +3874,8 @@
},
"Flags": "OW, Insert"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "REPLICAOF",
@@ -3806,7 +3907,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "ROLE",
@@ -3838,7 +3940,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "RPOPLPUSH",
@@ -3876,7 +3979,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "RPUSH",
@@ -3901,7 +4005,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "RPUSHX",
@@ -3926,7 +4031,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "RUNTXP",
@@ -3961,7 +4067,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SAVE",
@@ -3980,7 +4087,8 @@
"nondeterministic_output",
"request_policy:special",
"response_policy:special"
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "SCARD",
@@ -4005,7 +4113,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SCRIPT",
@@ -4074,7 +4183,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SDIFFSTORE",
@@ -4112,7 +4222,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SECONDARYOF",
@@ -4152,7 +4263,8 @@
"Notes": "RW and ACCESS due to the optional \u0060GET\u0060 argument",
"Flags": "RW, Access, Update, VariableFlags"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETBIT",
@@ -4177,7 +4289,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETEX",
@@ -4202,7 +4315,8 @@
},
"Flags": "OW, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETIFGREATER",
@@ -4226,7 +4340,8 @@
},
"Flags": "RW, Access, Update, VariableFlags"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETIFMATCH",
@@ -4250,7 +4365,8 @@
},
"Flags": "RW, Access, Update, VariableFlags"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETNX",
@@ -4275,7 +4391,8 @@
},
"Flags": "OW, Insert"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SETRANGE",
@@ -4300,7 +4417,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SINTER",
@@ -4328,7 +4446,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SINTERCARD",
@@ -4350,7 +4469,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SINTERSTORE",
@@ -4373,7 +4493,7 @@
"KeyStep": 1,
"Limit": 0
},
- "Flags": "OW, Update"
+ "Flags": "RW, Update"
},
{
"BeginSearch": {
@@ -4388,7 +4508,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SISMEMBER",
@@ -4413,7 +4534,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SECONDARYOF",
@@ -4497,7 +4619,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SMISMEMBER",
@@ -4522,7 +4645,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SMOVE",
@@ -4560,7 +4684,8 @@
},
"Flags": "RW, Insert"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SPOP",
@@ -4588,7 +4713,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SPUBLISH",
@@ -4641,7 +4767,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SREM",
@@ -4666,7 +4793,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SSCAN",
@@ -4694,7 +4822,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SSUBSCRIBE",
@@ -4744,7 +4873,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SUBSCRIBE",
@@ -4776,7 +4906,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Main"
},
{
"Command": "SUNION",
@@ -4804,7 +4935,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SUNIONSTORE",
@@ -4842,7 +4974,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "SWAPDB",
@@ -4887,7 +5020,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "TYPE",
@@ -4912,7 +5046,8 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "UNLINK",
@@ -4941,7 +5076,8 @@
},
"Flags": "RM, Delete"
}
- ]
+ ],
+ "StoreType": "All"
},
{
"Command": "UNSUBSCRIBE",
@@ -5055,7 +5191,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZCARD",
@@ -5080,17 +5217,18 @@
},
"Flags": "RO"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZCOUNT",
- "Name": "ZCOUNT",
- "Arity": 4,
- "Flags": "Fast, ReadOnly",
+ "Command": "ZCOLLECT",
+ "Name": "ZCOLLECT",
+ "Arity": 2,
+ "Flags": "Admin, Write",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "Fast, Read, SortedSet",
+ "AclCategories": "Admin, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5103,19 +5241,20 @@
"KeyStep": 1,
"Limit": 0
},
- "Flags": "RO, Access"
+ "Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZCOLLECT",
- "Name": "ZCOLLECT",
- "Arity": 2,
- "Flags": "Admin, Write",
+ "Command": "ZCOUNT",
+ "Name": "ZCOUNT",
+ "Arity": 4,
+ "Flags": "Fast, ReadOnly",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Write, Admin, Garnet",
+ "AclCategories": "Fast, Read, SortedSet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5128,9 +5267,10 @@
"KeyStep": 1,
"Limit": 0
},
- "Flags": "RW, Access, Update"
+ "Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZDIFF",
@@ -5152,7 +5292,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZDIFFSTORE",
@@ -5190,7 +5331,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZEXPIRE",
@@ -5200,7 +5342,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Write, Garnet",
+ "AclCategories": "Fast, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5215,7 +5357,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZEXPIREAT",
@@ -5225,7 +5368,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Write, Garnet",
+ "AclCategories": "Fast, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5240,7 +5383,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZEXPIRETIME",
@@ -5250,7 +5394,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Read, Garnet",
+ "AclCategories": "Fast, Read, SortedSet, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5265,7 +5409,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZINCRBY",
@@ -5290,7 +5435,8 @@
},
"Flags": "RW, Access, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZINTER",
@@ -5312,7 +5458,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZINTERCARD",
@@ -5334,7 +5481,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZINTERSTORE",
@@ -5372,7 +5520,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZLEXCOUNT",
@@ -5397,7 +5546,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZMPOP",
@@ -5419,7 +5569,8 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZMSCORE",
@@ -5444,7 +5595,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZPERSIST",
@@ -5454,7 +5606,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Write, Garnet",
+ "AclCategories": "Fast, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5469,7 +5621,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZPEXPIRE",
@@ -5479,7 +5632,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Write, Garnet",
+ "AclCategories": "Fast, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5494,7 +5647,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZPEXPIREAT",
@@ -5504,7 +5658,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Write, Garnet",
+ "AclCategories": "Fast, SortedSet, Write, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5519,7 +5673,8 @@
},
"Flags": "RW, Update"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZPEXPIRETIME",
@@ -5529,7 +5684,7 @@
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Read, Garnet",
+ "AclCategories": "Fast, Read, SortedSet, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5544,17 +5699,18 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZPTTL",
- "Name": "ZPTTL",
- "Arity": -5,
- "Flags": "Fast, ReadOnly",
+ "Command": "ZPOPMAX",
+ "Name": "ZPOPMAX",
+ "Arity": -2,
+ "Flags": "Fast, Write",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Read, Garnet",
+ "AclCategories": "Fast, SortedSet, Write",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5567,13 +5723,14 @@
"KeyStep": 1,
"Limit": 0
},
- "Flags": "RO, Access"
+ "Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZPOPMAX",
- "Name": "ZPOPMAX",
+ "Command": "ZPOPMIN",
+ "Name": "ZPOPMIN",
"Arity": -2,
"Flags": "Fast, Write",
"FirstKey": 1,
@@ -5594,17 +5751,18 @@
},
"Flags": "RW, Access, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZPOPMIN",
- "Name": "ZPOPMIN",
- "Arity": -2,
- "Flags": "Fast, Write",
+ "Command": "ZPTTL",
+ "Name": "ZPTTL",
+ "Arity": -5,
+ "Flags": "Fast, ReadOnly",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "Fast, SortedSet, Write",
+ "AclCategories": "Fast, Read, SortedSet, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -5617,9 +5775,10 @@
"KeyStep": 1,
"Limit": 0
},
- "Flags": "RW, Access, Delete"
+ "Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANDMEMBER",
@@ -5647,7 +5806,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANGE",
@@ -5672,7 +5832,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANGEBYLEX",
@@ -5697,7 +5858,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANGEBYSCORE",
@@ -5722,7 +5884,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANGESTORE",
@@ -5760,7 +5923,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZRANK",
@@ -5785,7 +5949,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREM",
@@ -5810,7 +5975,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREMRANGEBYLEX",
@@ -5835,7 +6001,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREMRANGEBYRANK",
@@ -5860,7 +6027,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREMRANGEBYSCORE",
@@ -5885,7 +6053,8 @@
},
"Flags": "RW, Delete"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREVRANGE",
@@ -5910,7 +6079,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREVRANGEBYLEX",
@@ -5935,7 +6105,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREVRANGEBYSCORE",
@@ -5960,7 +6131,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZREVRANK",
@@ -5985,7 +6157,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZSCAN",
@@ -6013,17 +6186,18 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZTTL",
- "Name": "ZTTL",
- "Arity": -5,
+ "Command": "ZSCORE",
+ "Name": "ZSCORE",
+ "Arity": 3,
"Flags": "Fast, ReadOnly",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "SortedSet, Fast, Read, Garnet",
+ "AclCategories": "Fast, Read, SortedSet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -6038,17 +6212,18 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
- "Command": "ZSCORE",
- "Name": "ZSCORE",
- "Arity": 3,
+ "Command": "ZTTL",
+ "Name": "ZTTL",
+ "Arity": -5,
"Flags": "Fast, ReadOnly",
"FirstKey": 1,
"LastKey": 1,
"Step": 1,
- "AclCategories": "Fast, Read, SortedSet",
+ "AclCategories": "Fast, Read, SortedSet, Garnet",
"KeySpecifications": [
{
"BeginSearch": {
@@ -6063,7 +6238,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZUNION",
@@ -6085,7 +6261,8 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
},
{
"Command": "ZUNIONSTORE",
@@ -6123,6 +6300,7 @@
},
"Flags": "RO, Access"
}
- ]
+ ],
+ "StoreType": "Object"
}
]
\ No newline at end of file
diff --git a/libs/server/AOF/AofEntryType.cs b/libs/server/AOF/AofEntryType.cs
index bccbdd6af07..1a3369166d9 100644
--- a/libs/server/AOF/AofEntryType.cs
+++ b/libs/server/AOF/AofEntryType.cs
@@ -79,6 +79,23 @@ public enum AofEntryType : byte
///
FlushDb = 0x61,
+ ///
+ /// Unified store upsert sting
+ ///
+ UnifiedStoreStringUpsert = 0x70,
+ ///
+ /// Unified store upsert object
+ ///
+ UnifiedStoreObjectUpsert = 0x71,
+ ///
+ /// Unified store RMW
+ ///
+ UnifiedStoreRMW = 0x72,
+ ///
+ /// Unified store delete
+ ///
+ UnifiedStoreDelete = 0x73,
+
#region Deprecated markers
///
/// Deprecated with unified checkpointing: Checkpoint for object store start
diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs
index 679c83bd51c..73f7468d0ff 100644
--- a/libs/server/AOF/AofProcessor.cs
+++ b/libs/server/AOF/AofProcessor.cs
@@ -13,11 +13,8 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Wrapper for store and store-specific information
@@ -29,6 +26,7 @@ public sealed unsafe partial class AofProcessor
private readonly RawStringInput storeInput;
private readonly ObjectInput objectStoreInput;
+ private readonly UnifiedStoreInput unifiedStoreInput;
private readonly CustomProcedureInput customProcInput;
private readonly SessionParseState parseState;
@@ -42,12 +40,17 @@ public sealed unsafe partial class AofProcessor
///
/// Session for main store
///
- BasicContext basicContext;
+ BasicContext basicContext;
///
/// Session for object store
///
- BasicContext objectStoreBasicContext;
+ BasicContext objectStoreBasicContext;
+
+ ///
+ /// Session for unified store
+ ///
+ BasicContext unifiedStoreBasicContext;
readonly Dictionary> inflightTxns;
readonly byte[] buffer;
@@ -80,6 +83,7 @@ public AofProcessor(
parseState.Initialize();
storeInput.parseState = parseState;
objectStoreInput.parseState = parseState;
+ unifiedStoreInput.parseState = parseState;
customProcInput.parseState = parseState;
inflightTxns = [];
@@ -99,6 +103,7 @@ public void Dispose()
{
dbSession.StorageSession.basicContext.Session?.Dispose();
dbSession.StorageSession.objectStoreBasicContext.Session?.Dispose();
+ dbSession.StorageSession.unifiedStoreBasicContext.Session?.Dispose();
}
handle.Free();
@@ -295,7 +300,7 @@ public unsafe void ProcessAofRecordInternal(byte* ptr, int length, bool asReplic
case AofEntryType.ObjectStoreStreamingCheckpointStartCommit:
Debug.Assert(storeWrapper.serverOptions.ReplicaDisklessSync);
if (asReplica && header.storeVersion > storeWrapper.store.CurrentVersion)
- storeWrapper.objectStore.SetVersion(header.storeVersion);
+ storeWrapper.store.SetVersion(header.storeVersion);
break;
case AofEntryType.FlushAll:
storeWrapper.FlushAllDatabases(unsafeTruncateLog: header.unsafeTruncateLog == 1);
@@ -354,6 +359,18 @@ private unsafe bool ReplayOp(byte* entryPtr, int length, bool replayAsReplica)
case AofEntryType.ObjectStoreDelete:
ObjectStoreDelete(objectStoreBasicContext, entryPtr);
break;
+ case AofEntryType.UnifiedStoreRMW:
+ UnifiedStoreRMW(unifiedStoreBasicContext, unifiedStoreInput, entryPtr, bufferPtr, buffer.Length);
+ break;
+ case AofEntryType.UnifiedStoreStringUpsert:
+ UnifiedStoreStringUpsert(unifiedStoreBasicContext, unifiedStoreInput, entryPtr, bufferPtr, buffer.Length);
+ break;
+ case AofEntryType.UnifiedStoreObjectUpsert:
+ UnifiedStoreObjectUpsert(unifiedStoreBasicContext, storeWrapper.GarnetObjectSerializer, entryPtr, bufferPtr, buffer.Length);
+ break;
+ case AofEntryType.UnifiedStoreDelete:
+ UnifiedStoreDelete(unifiedStoreBasicContext, entryPtr);
+ break;
case AofEntryType.StoredProcedure:
RunStoredProc(header.procedureId, customProcInput, entryPtr);
break;
@@ -372,7 +389,7 @@ void RunStoredProc(byte id, CustomProcedureInput customProcInput, byte* ptr)
customProcInput.DeserializeFrom(curr);
// Run the stored procedure with the reconstructed input
- respServerSession.RunTransactionProc(id, ref customProcInput, ref output);
+ respServerSession.RunTransactionProc(id, ref customProcInput, ref output, isRecovering: true);
}
}
@@ -388,16 +405,14 @@ private void SwitchActiveDatabaseContext(GarnetDatabase db, bool initialSetup =
// Switch the storage context to match the session, if necessary
if (this.activeDbId != db.Id || initialSetup)
{
- var session = respServerSession.storageSession.basicContext.Session;
- basicContext = session.BasicContext;
- var objectStoreSession = respServerSession.storageSession.objectStoreBasicContext.Session;
- if (objectStoreSession is not null)
- objectStoreBasicContext = objectStoreSession.BasicContext;
+ basicContext = respServerSession.storageSession.basicContext.Session.BasicContext;
+ objectStoreBasicContext = respServerSession.storageSession.objectStoreBasicContext.Session.BasicContext;
+ unifiedStoreBasicContext = respServerSession.storageSession.unifiedStoreBasicContext.Session.BasicContext;
this.activeDbId = db.Id;
}
}
- static void StoreUpsert(BasicContext basicContext,
+ static void StoreUpsert(BasicContext basicContext,
RawStringInput storeInput, byte* ptr)
{
var curr = ptr + sizeof(AofHeader);
@@ -417,7 +432,7 @@ static void StoreUpsert(BasicContext basicContext, RawStringInput storeInput, byte* ptr)
+ static void StoreRMW(BasicContext basicContext, RawStringInput storeInput, byte* ptr)
{
var curr = ptr + sizeof(AofHeader);
var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr);
@@ -436,13 +451,13 @@ static void StoreRMW(BasicContext basicContext, byte* ptr)
+ static void StoreDelete(BasicContext basicContext, byte* ptr)
{
var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader));
basicContext.Delete(key);
}
- static void ObjectStoreUpsert(BasicContext basicContext,
+ static void ObjectStoreUpsert(BasicContext basicContext,
GarnetObjectSerializer garnetObjectSerializer, byte* ptr, byte* outputPtr, int outputLength)
{
var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader));
@@ -459,7 +474,7 @@ static void ObjectStoreUpsert(BasicContext basicContext,
+ static void ObjectStoreRMW(BasicContext basicContext,
ObjectInput objectStoreInput, byte* ptr, byte* outputPtr, int outputLength)
{
var curr = ptr + sizeof(AofHeader);
@@ -480,7 +495,73 @@ static void ObjectStoreRMW(BasicContext basicContext, byte* ptr)
+ static void ObjectStoreDelete(BasicContext basicContext, byte* ptr)
+ {
+ var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader));
+ basicContext.Delete(key);
+ }
+
+ static void UnifiedStoreStringUpsert(BasicContext basicContext,
+ UnifiedStoreInput storeInput, byte* ptr, byte* outputPtr, int outputLength)
+ {
+ var curr = ptr + sizeof(AofHeader);
+ var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr);
+ curr += key.TotalSize;
+
+ var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr);
+ curr += value.TotalSize;
+
+ // Reconstructing UnifiedStoreInput
+
+ // input
+ _ = storeInput.DeserializeFrom(curr);
+
+ var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength);
+ basicContext.Upsert(key.ReadOnlySpan, ref storeInput, value.ReadOnlySpan, ref output);
+ if (!output.SpanByteAndMemory.IsSpanByte)
+ output.SpanByteAndMemory.Memory.Dispose();
+ }
+
+ static void UnifiedStoreObjectUpsert(BasicContext basicContext,
+ GarnetObjectSerializer garnetObjectSerializer, byte* ptr, byte* outputPtr, int outputLength)
+ {
+ var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader));
+
+ var value = PinnedSpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader) + key.TotalSize);
+ var valB = garnetObjectSerializer.Deserialize(value.ToArray());
+
+ // input
+ // TODOMigrate: _ = unifiedStoreInput.DeserializeFrom(curr); // TODO - need to serialize this as well
+
+ var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength);
+ basicContext.Upsert(key.ReadOnlySpan, valB);
+ if (!output.SpanByteAndMemory.IsSpanByte)
+ output.SpanByteAndMemory.Memory.Dispose();
+ }
+
+ static void UnifiedStoreRMW(BasicContext basicContext,
+ UnifiedStoreInput unifiedStoreInput, byte* ptr, byte* outputPtr, int outputLength)
+ {
+ var curr = ptr + sizeof(AofHeader);
+ var key = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr);
+ curr += key.TotalSize;
+
+ // Reconstructing UnifiedStoreInput
+
+ // input
+ _ = unifiedStoreInput.DeserializeFrom(curr);
+
+ // Call RMW with the reconstructed key & UnifiedStoreInput
+ var output = GarnetUnifiedStoreOutput.FromPinnedPointer(outputPtr, outputLength);
+ if (basicContext.RMW(key.ReadOnlySpan, ref unifiedStoreInput, ref output).IsPending)
+ basicContext.CompletePending(true);
+
+ if (!output.SpanByteAndMemory.IsSpanByte)
+ output.SpanByteAndMemory.Memory.Dispose();
+ }
+
+ static void UnifiedStoreDelete(
+ BasicContext basicContext, byte* ptr)
{
var key = SpanByte.FromLengthPrefixedPinnedPointer(ptr + sizeof(AofHeader));
basicContext.Delete(key);
@@ -513,42 +594,9 @@ bool BufferNewVersionRecord(AofHeader header, byte* entryPtr, int length)
}
bool IsOldVersionRecord(AofHeader header)
- {
- var storeType = ToAofStoreType(header.opType);
-
- return storeType switch
- {
- AofStoreType.MainStoreType => header.storeVersion < storeWrapper.store.CurrentVersion,
- AofStoreType.ObjectStoreType => header.storeVersion < storeWrapper.objectStore.CurrentVersion,
- AofStoreType.TxnType => header.storeVersion < storeWrapper.objectStore.CurrentVersion,
- _ => throw new GarnetException($"Unexpected AOF header store type {storeType}"),
- };
- }
+ => header.storeVersion < storeWrapper.store.CurrentVersion;
bool IsNewVersionRecord(AofHeader header)
- {
- var storeType = ToAofStoreType(header.opType);
- return storeType switch
- {
- AofStoreType.MainStoreType => header.storeVersion > storeWrapper.store.CurrentVersion,
- AofStoreType.ObjectStoreType => header.storeVersion > storeWrapper.objectStore.CurrentVersion,
- AofStoreType.TxnType => header.storeVersion > storeWrapper.objectStore.CurrentVersion,
- _ => throw new GarnetException($"Unknown AOF header store type {storeType}"),
- };
- }
-
- static AofStoreType ToAofStoreType(AofEntryType type)
- {
- return type switch
- {
- AofEntryType.StoreUpsert or AofEntryType.StoreRMW or AofEntryType.StoreDelete => AofStoreType.MainStoreType,
- AofEntryType.ObjectStoreUpsert or AofEntryType.ObjectStoreRMW or AofEntryType.ObjectStoreDelete => AofStoreType.ObjectStoreType,
- AofEntryType.TxnStart or AofEntryType.TxnCommit or AofEntryType.TxnAbort or AofEntryType.StoredProcedure => AofStoreType.TxnType,
- AofEntryType.CheckpointStartCommit or AofEntryType.ObjectStoreCheckpointStartCommit or AofEntryType.MainStoreStreamingCheckpointStartCommit or AofEntryType.ObjectStoreStreamingCheckpointStartCommit => AofStoreType.CheckpointType,
- AofEntryType.CheckpointEndCommit or AofEntryType.ObjectStoreCheckpointEndCommit or AofEntryType.MainStoreStreamingCheckpointEndCommit or AofEntryType.ObjectStoreStreamingCheckpointEndCommit => AofStoreType.CheckpointType,
- AofEntryType.FlushAll or AofEntryType.FlushDb => AofStoreType.FlushDbType,
- _ => throw new GarnetException($"Conversion to AofStoreType not possible for {type}"),
- };
- }
+ => header.storeVersion > storeWrapper.store.CurrentVersion;
}
}
\ No newline at end of file
diff --git a/libs/server/API/GarnetApi.cs b/libs/server/API/GarnetApi.cs
index 99886b74fd5..617eac186eb 100644
--- a/libs/server/API/GarnetApi.cs
+++ b/libs/server/API/GarnetApi.cs
@@ -8,30 +8,30 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
// See TransactionManager.cs for aliases BasicGarnetApi and TransactionalGarnetApi
///
/// Garnet API implementation
///
- public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
- where TContext : ITsavoriteContext
- where TObjectContext : ITsavoriteContext
+ public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
+ where TContext : ITsavoriteContext
+ where TObjectContext : ITsavoriteContext
+ where TUnifiedContext : ITsavoriteContext
{
readonly StorageSession storageSession;
TContext context;
TObjectContext objectContext;
+ TUnifiedContext unifiedContext;
- internal GarnetApi(StorageSession storageSession, TContext context, TObjectContext objectContext)
+ internal GarnetApi(StorageSession storageSession, TContext context, TObjectContext objectContext, TUnifiedContext unifiedContext)
{
this.storageSession = storageSession;
this.context = context;
this.objectContext = objectContext;
+ this.unifiedContext = unifiedContext;
}
#region WATCH
@@ -87,30 +87,6 @@ public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref S
=> storageSession.GETRANGE(key, ref input, ref output, ref context);
#endregion
- #region TTL
-
- ///
- public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- => storageSession.TTL(key, storeType, ref output, ref context, ref objectContext);
-
- ///
- public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- => storageSession.TTL(key, storeType, ref output, ref context, ref objectContext, milliseconds: true);
-
- #endregion
-
- #region EXPIRETIME
-
- ///
- public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- => storageSession.EXPIRETIME(key, storeType, ref output, ref context, ref objectContext);
-
- ///
- public GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- => storageSession.EXPIRETIME(key, storeType, ref output, ref context, ref objectContext, milliseconds: true);
-
- #endregion
-
#region SET
///
public GarnetStatus SET(PinnedSpanByte key, PinnedSpanByte value)
@@ -141,9 +117,15 @@ public GarnetStatus SET(PinnedSpanByte key, IGarnetObject value)
=> storageSession.SET(key, value, ref objectContext);
///
- public GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType)
+ public GarnetStatus SET_Main(in TSourceLogRecord srcLogRecord)
where TSourceLogRecord : ISourceLogRecord
- => storageSession.SET(in srcLogRecord, storeType, ref context, ref objectContext);
+ => storageSession.SET_Main(in srcLogRecord, ref context);
+
+ ///
+ public GarnetStatus SET_Object(in TSourceLogRecord srcLogRecord)
+ where TSourceLogRecord : ISourceLogRecord
+ => storageSession.SET_Object(in srcLogRecord, ref objectContext);
+
#endregion
#region SETEX
@@ -193,44 +175,6 @@ public GarnetStatus RENAMENX(PinnedSpanByte oldKey, PinnedSpanByte newKey, out i
=> storageSession.RENAMENX(oldKey, newKey, storeType, out result, withEtag);
#endregion
- #region EXISTS
- ///
- public GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All)
- => storageSession.EXISTS(key, storeType, ref context, ref objectContext);
- #endregion
-
- #region EXPIRE
- ///
- public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All)
- => storageSession.EXPIRE(key, ref input, out timeoutSet, storeType, ref context, ref objectContext);
-
- ///
- public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None)
- => storageSession.EXPIRE(key, expiryMs, out timeoutSet, storeType, expireOption, ref context, ref objectContext);
-
- ///
- public GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None)
- => storageSession.EXPIRE(key, expiry, out timeoutSet, storeType, expireOption, ref context, ref objectContext);
- #endregion
-
- #region EXPIREAT
-
- ///
- public GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None)
- => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectContext);
-
- ///
- public GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None)
- => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, storeType, expireOption, ref context, ref objectContext, milliseconds: true);
-
- #endregion
-
- #region PERSIST
- ///
- public unsafe GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType = StoreType.All)
- => storageSession.PERSIST(key, storeType, ref context, ref objectContext);
- #endregion
-
#region Increment (INCR, INCRBY, DECR, DECRBY)
///
public GarnetStatus Increment(PinnedSpanByte key, ref RawStringInput input, ref PinnedSpanByte output)
@@ -290,34 +234,12 @@ public GarnetStatus IncrementByFloat(PinnedSpanByte key, out double output, doub
}
#endregion
- #region DELETE
- ///
- public GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All)
- => storageSession.DELETE(key, storeType, ref context, ref objectContext);
- #endregion
-
#region GETDEL
///
public GarnetStatus GETDEL(PinnedSpanByte key, ref SpanByteAndMemory output)
=> storageSession.GETDEL(key, ref output, ref context);
#endregion
- #region TYPE
-
- ///
- public GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName)
- => storageSession.GetKeyType(key, out typeName, ref context, ref objectContext);
-
- #endregion
-
- #region MEMORY
-
- ///
- public GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, int samples = 0)
- => storageSession.MemoryUsageForKey(key, out memoryUsage, ref context, ref objectContext, samples);
-
- #endregion
-
#region Advanced ops
///
public GarnetStatus RMW_MainStore(PinnedSpanByte key, ref RawStringInput input, ref SpanByteAndMemory output)
diff --git a/libs/server/API/GarnetApiObjectCommands.cs b/libs/server/API/GarnetApiObjectCommands.cs
index 82ed66e00f6..2bff58addc1 100644
--- a/libs/server/API/GarnetApiObjectCommands.cs
+++ b/libs/server/API/GarnetApiObjectCommands.cs
@@ -7,18 +7,16 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Garnet API implementation
///
- public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
- where TContext : ITsavoriteContext
- where TObjectContext : ITsavoriteContext
+ public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
+ where TContext : ITsavoriteContext
+ where TObjectContext : ITsavoriteContext
+ where TUnifiedContext : ITsavoriteContext
{
#region SortedSet Methods
@@ -47,7 +45,7 @@ public GarnetStatus SortedSetRemove(PinnedSpanByte key, PinnedSpanByte[] members
=> storageSession.SortedSetRemove(key, members, out zaddCount, ref objectContext);
///
- public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SortedSetRemove(key, ref input, out output, ref objectContext);
///
@@ -55,7 +53,7 @@ public GarnetStatus SortedSetLength(PinnedSpanByte key, out int len)
=> storageSession.SortedSetLength(key, out len, ref objectContext);
///
- public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SortedSetLength(key, ref input, out output, ref objectContext);
///
@@ -91,11 +89,11 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, re
=> storageSession.SortedSetCount(key, ref input, ref output, ref objectContext);
///
- public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SortedSetLengthByValue(key, ref input, out output, ref objectContext);
///
- public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SortedSetRemoveRangeByLex(key, ref input, out output, ref objectContext);
///
@@ -241,7 +239,7 @@ public GarnetStatus ListRightPush(PinnedSpanByte key, PinnedSpanByte[] elements,
=> storageSession.ListPush(key, elements, whenExists ? ListOperation.RPUSHX : ListOperation.RPUSH, out itemsCount, ref objectContext);
///
- public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.ListPush(key, ref input, out output, ref objectContext);
///
@@ -253,7 +251,7 @@ public GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte element, out
=> storageSession.ListPush(key, element, onlyWhenExists ? ListOperation.LPUSHX : ListOperation.LPUSH, out count, ref objectContext);
///
- public GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.ListPush(key, ref input, out output, ref objectContext);
///
@@ -299,7 +297,7 @@ public GarnetStatus ListLength(PinnedSpanByte key, out int count)
=> storageSession.ListLength(key, ref objectContext, out count);
///
- public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.ListLength(key, ref input, out output, ref objectContext);
///
@@ -319,7 +317,7 @@ public GarnetStatus ListRange(PinnedSpanByte key, ref ObjectInput input, ref Gar
=> storageSession.ListRange(key, ref input, ref output, ref objectContext);
///
- public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.ListInsert(key, ref input, out output, ref objectContext);
///
@@ -327,7 +325,7 @@ public GarnetStatus ListIndex(PinnedSpanByte key, ref ObjectInput input, ref Gar
=> storageSession.ListIndex(key, ref input, ref output, ref objectContext);
///
- public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.ListRemove(key, ref input, out output, ref objectContext);
///
@@ -347,7 +345,7 @@ public GarnetStatus SetAdd(PinnedSpanByte key, PinnedSpanByte[] members, out int
=> storageSession.SetAdd(key, members, out saddCount, ref objectContext);
///
- public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SetAdd(key, ref input, out output, ref objectContext);
///
@@ -359,7 +357,7 @@ public GarnetStatus SetRemove(PinnedSpanByte key, PinnedSpanByte[] members, out
=> storageSession.SetRemove(key, members, out sremCount, ref objectContext);
///
- public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SetRemove(key, ref input, out output, ref objectContext);
///
@@ -367,7 +365,7 @@ public GarnetStatus SetLength(PinnedSpanByte key, out int count)
=> storageSession.SetLength(key, out count, ref objectContext);
///
- public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.SetLength(key, ref input, out output, ref objectContext);
///
@@ -454,7 +452,7 @@ public GarnetStatus HashSet(PinnedSpanByte key, (PinnedSpanByte field, PinnedSpa
=> storageSession.HashSet(key, elements, out count, ref objectContext);
///
- public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.HashSet(key, ref input, out output, ref objectContext);
///
@@ -494,11 +492,11 @@ public GarnetStatus HashLength(PinnedSpanByte key, out int count)
=> storageSession.HashLength(key, out count, ref objectContext);
///
- public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.HashLength(key, ref input, out output, ref objectContext);
///
- public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.HashStrLength(key, ref input, out output, ref objectContext);
///
@@ -506,7 +504,7 @@ public GarnetStatus HashExists(PinnedSpanByte key, PinnedSpanByte field, out boo
=> storageSession.HashExists(key, field, out exists, ref objectContext);
///
- public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.HashExists(key, ref input, out output, ref objectContext);
///
@@ -522,7 +520,7 @@ public GarnetStatus HashRandomField(PinnedSpanByte key, ref ObjectInput input, r
=> storageSession.HashRandomField(key, ref input, ref output, ref objectContext);
///
- public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
=> storageSession.HashDelete(key, ref input, out output, ref objectContext);
///
@@ -534,7 +532,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref Garn
=> storageSession.HashVals(key, ref input, ref output, ref objectContext);
///
- public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output)
+ public GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out OutputHeader output)
=> storageSession.HashIncrement(key, input, out output, ref objectContext);
///
diff --git a/libs/server/API/GarnetApiUnifiedCommands.cs b/libs/server/API/GarnetApiUnifiedCommands.cs
new file mode 100644
index 00000000000..8f86389a373
--- /dev/null
+++ b/libs/server/API/GarnetApiUnifiedCommands.cs
@@ -0,0 +1,108 @@
+// Copyright (c) Microsoft Corporation.
+// Licensed under the MIT license.
+
+using System;
+using Tsavorite.core;
+
+namespace Garnet.server
+{
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
+
+ ///
+ /// Garnet API implementation
+ ///
+ public partial struct GarnetApi : IGarnetApi, IGarnetWatchApi
+ where TContext : ITsavoriteContext
+ where TObjectContext : ITsavoriteContext
+ where TUnifiedContext : ITsavoriteContext
+ {
+ #region MEMORY
+
+ ///
+ public GarnetStatus MEMORYUSAGE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ #endregion
+
+ #region TYPE
+
+ ///
+ public GarnetStatus TYPE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ #endregion
+
+ #region TTL
+
+ ///
+ public GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ #endregion
+
+ #region EXPIRETIME
+
+ ///
+ public GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ #endregion
+
+ #region EXISTS
+
+ ///
+ public GarnetStatus EXISTS(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.Read_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ ///
+ public GarnetStatus EXISTS(PinnedSpanByte key)
+ => storageSession.EXISTS(key, ref unifiedContext);
+
+ #endregion
+
+ #region DELETE
+
+ ///
+ public GarnetStatus DELETE(PinnedSpanByte key)
+ => storageSession.DELETE(key, ref unifiedContext);
+
+ #endregion
+
+ #region EXPIRE
+
+ ///
+ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ ///
+ public unsafe GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None)
+ => storageSession.EXPIRE(key, expiryMs, out timeoutSet, expireOption, ref unifiedContext);
+
+ ///
+ public GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None)
+ => storageSession.EXPIRE(key, expiry, out timeoutSet, expireOption, ref unifiedContext);
+
+ #endregion
+
+ #region EXPIREAT
+
+ ///
+ public GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None)
+ => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, expireOption, ref unifiedContext);
+
+ ///
+ public GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None)
+ => storageSession.EXPIREAT(key, expiryTimestamp, out timeoutSet, expireOption, ref unifiedContext, milliseconds: true);
+
+ #endregion
+
+ #region PERSIST
+
+ ///
+ public unsafe GarnetStatus PERSIST(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
+ => storageSession.RMW_UnifiedStore(key, ref input, ref output, ref unifiedContext);
+
+ #endregion
+ }
+}
\ No newline at end of file
diff --git a/libs/server/API/GarnetWatchApi.cs b/libs/server/API/GarnetWatchApi.cs
index 2452019132c..617ea5849ba 100644
--- a/libs/server/API/GarnetWatchApi.cs
+++ b/libs/server/API/GarnetWatchApi.cs
@@ -70,17 +70,10 @@ public GarnetStatus GETRANGE(PinnedSpanByte key, ref RawStringInput input, ref S
#region TTL
///
- public GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
+ public GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
{
- garnetApi.WATCH(key, storeType);
- return garnetApi.TTL(key, storeType, ref output);
- }
-
- ///
- public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- {
- garnetApi.WATCH(key, storeType);
- return garnetApi.PTTL(key, storeType, ref output);
+ garnetApi.WATCH(key, StoreType.All);
+ return garnetApi.TTL(key, ref input, ref output);
}
#endregion
@@ -88,17 +81,10 @@ public GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAn
#region EXPIRETIME
///
- public GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
+ public GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output)
{
- garnetApi.WATCH(key, storeType);
- return garnetApi.EXPIRETIME(key, storeType, ref output);
- }
-
- ///
- public GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output)
- {
- garnetApi.WATCH(key, storeType);
- return garnetApi.PEXPIRETIME(key, storeType, ref output);
+ garnetApi.WATCH(key, StoreType.All);
+ return garnetApi.EXPIRETIME(key, ref input, ref output);
}
#endregion
@@ -113,7 +99,7 @@ public GarnetStatus SortedSetLength(PinnedSpanByte key, out int zcardCount)
}
///
- public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.SortedSetLength(key, ref input, out output);
@@ -134,7 +120,7 @@ public GarnetStatus SortedSetCount(PinnedSpanByte key, ref ObjectInput input, re
}
///
- public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.SortedSetLengthByValue(key, ref input, out output);
@@ -277,7 +263,7 @@ public GarnetStatus ListLength(PinnedSpanByte key, out int count)
}
///
- public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.ListLength(key, ref input, out output);
@@ -309,7 +295,7 @@ public GarnetStatus SetLength(PinnedSpanByte key, out int scardCount)
}
///
- public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.SetLength(key, ref input, out output);
@@ -468,14 +454,14 @@ public GarnetStatus HashGetMultiple(PinnedSpanByte key, ref ObjectInput input, r
}
///
- public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.HashStrLength(key, ref input, out output);
}
///
- public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.HashExists(key, ref input, out output);
@@ -496,7 +482,7 @@ public GarnetStatus HashVals(PinnedSpanByte key, ref ObjectInput input, ref Garn
}
///
- public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output)
+ public GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output)
{
garnetApi.WATCH(key, StoreType.Object);
return garnetApi.HashLength(key, ref input, out output);
diff --git a/libs/server/API/IGarnetApi.cs b/libs/server/API/IGarnetApi.cs
index 5d27b7d6591..72726bb0d35 100644
--- a/libs/server/API/IGarnetApi.cs
+++ b/libs/server/API/IGarnetApi.cs
@@ -59,10 +59,15 @@ public interface IGarnetApi : IGarnetReadApi, IGarnetAdvancedApi
GarnetStatus SET(PinnedSpanByte key, IGarnetObject value);
///
- /// SET
+ /// SET in main store
///
- GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType storeType)
- where TSourceLogRecord : ISourceLogRecord;
+ GarnetStatus SET_Main(in TSourceLogRecord srcLogRecord) where TSourceLogRecord : ISourceLogRecord;
+
+ ///
+ /// SET in object store
+ ///
+ GarnetStatus SET_Object(in TSourceLogRecord srcLogRecord) where TSourceLogRecord : ISourceLogRecord;
+
#endregion
#region SETEX
@@ -149,13 +154,23 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
#endregion
#region EXISTS
+
///
/// EXISTS
///
///
- ///
+ ///
+ ///
+ ///
+ GarnetStatus EXISTS(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
+
+ ///
+ /// EXISTS
+ ///
+ /// Key
///
- GarnetStatus EXISTS(PinnedSpanByte key, StoreType storeType = StoreType.All);
+ GarnetStatus EXISTS(PinnedSpanByte key);
+
#endregion
#region EXPIRE
@@ -165,20 +180,18 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// Key
/// Expiry in milliseconds, formatted as ASCII digits
/// Whether timeout was set by the call
- /// Store type: main, object, or both
/// Expire option
///
- GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None);
+ GarnetStatus EXPIRE(PinnedSpanByte key, PinnedSpanByte expiryMs, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None);
///
/// Set a timeout on key using a timeSpan in seconds
///
/// Key
///
- /// Whether timeout was set by the call
- /// Store type: main, object, or both
+ ///
///
- GarnetStatus EXPIRE(PinnedSpanByte key, ref RawStringInput input, out bool timeoutSet, StoreType storeType = StoreType.All);
+ GarnetStatus EXPIRE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
///
/// Set a timeout on key using a timeSpan in seconds
@@ -186,10 +199,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// Key
/// Expiry in TimeSpan
/// Whether timeout was set by the call
- /// Store type: main, object, or both
/// Expire option
///
- GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None);
+ GarnetStatus EXPIRE(PinnedSpanByte key, TimeSpan expiry, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None);
#endregion
#region EXPIREAT
@@ -200,10 +212,9 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// Key
/// Absolute Unix timestamp in seconds
/// Whether timeout was set by the call
- /// Store type: main, object, or both
/// Expire option
///
- GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None);
+ GarnetStatus EXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None);
///
/// Set a timeout on key using absolute Unix timestamp (seconds since January 1, 1970) in milliseconds
@@ -211,21 +222,22 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// Key
/// Absolute Unix timestamp in milliseconds
/// Whether timeout was set by the call
- /// Store type: main, object, or both
/// Expire option
///
- GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, StoreType storeType = StoreType.All, ExpireOption expireOption = ExpireOption.None);
+ GarnetStatus PEXPIREAT(PinnedSpanByte key, long expiryTimestamp, out bool timeoutSet, ExpireOption expireOption = ExpireOption.None);
#endregion
#region PERSIST
+
///
/// PERSIST
///
/// Key
- /// Store type: main, object, or both
+ ///
+ ///
///
- GarnetStatus PERSIST(PinnedSpanByte key, StoreType storeType = StoreType.All);
+ GarnetStatus PERSIST(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
#endregion
#region Increment (INCR, INCRBY, DECR, DECRBY)
@@ -276,13 +288,14 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
#endregion
#region DELETE
+
///
- /// DELETE
+ /// Deletes a key from the unified store
///
///
- ///
///
- GarnetStatus DELETE(PinnedSpanByte key, StoreType storeType = StoreType.All);
+ GarnetStatus DELETE(PinnedSpanByte key);
+
#endregion
#region GETDEL
@@ -302,9 +315,10 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// string, list, set, zset, and hash.
///
///
- ///
+ ///
+ ///
///
- GarnetStatus GetKeyType(PinnedSpanByte key, out string typeName);
+ GarnetStatus TYPE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
#endregion
@@ -314,10 +328,10 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
/// Gets the number of bytes that a key and its value require to be stored in RAM.
///
/// Name of the key or object to get the memory usage
- /// The value in bytes the key or object is using
- /// Number of sampled nested values
+ ///
+ ///
/// GarnetStatus
- GarnetStatus MemoryUsageForKey(PinnedSpanByte key, out long memoryUsage, int samples = 0);
+ GarnetStatus MEMORYUSAGE(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
#endregion
@@ -386,7 +400,7 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
///
///
///
- GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SortedSetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Removes all elements in the sorted set between the
@@ -396,7 +410,7 @@ GarnetStatus SET(in TSourceLogRecord srcLogRecord, StoreType s
///
///
///
- GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SortedSetRemoveRangeByLex(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Removes and returns the first element from the sorted set stored at key,
@@ -637,7 +651,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SetAdd(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Removes the specified member from the set.
@@ -670,7 +684,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SetRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Removes and returns one random member from the set at key.
@@ -769,13 +783,13 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
GarnetStatus ListPosition(PinnedSpanByte key, ref ObjectInput input, ref GarnetObjectStoreOutput output);
///
- /// ListLeftPush ArgSlice version with ObjectOutputHeader output
+ /// ListLeftPush ArgSlice version with OutputHeader output
///
///
///
///
///
- GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus ListLeftPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// ListLeftPush ArgSlice version, one element
@@ -798,13 +812,13 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
GarnetStatus ListLeftPush(PinnedSpanByte key, PinnedSpanByte[] elements, out int count, bool whenExists = false);
///
- /// ListRightPush ArgSlice version with ObjectOutputHeader output
+ /// ListRightPush ArgSlice version with OutputHeader output
///
///
///
///
///
- public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ public GarnetStatus ListRightPush(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// ListRightPush ArgSlice version, one element
@@ -941,7 +955,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus ListInsert(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Removes the first count occurrences of elements equal to element from the list.
@@ -950,7 +964,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus ListRemove(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Sets the list element at index to element.
@@ -995,7 +1009,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus HashSet(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Set only if field does not yet exist. If key does not exist, a new key holding a hash is created.
@@ -1034,7 +1048,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus HashDelete(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Increments the number stored at field in the hash key by increment parameter.
@@ -1043,7 +1057,7 @@ GarnetStatus GeoSearchStore(PinnedSpanByte key, PinnedSpanByte destinationKey, r
///
///
///
- GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out ObjectOutputHeader output);
+ GarnetStatus HashIncrement(PinnedSpanByte key, PinnedSpanByte input, out OutputHeader output);
///
/// Increments the number stored at field representing a floating point value
@@ -1241,19 +1255,10 @@ public interface IGarnetReadApi
/// Returns the remaining time to live in seconds of a key that has a timeout.
///
/// The key to return the remaining time to live in the store
- /// The store type to operate on.
- /// The span to allocate the output of the operation.
- ///
- GarnetStatus TTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output);
-
- ///
- /// Returns the remaining time to live in milliseconds of a key that has a timeout.
- ///
- /// The key to return the remaining time to live in the store.
- /// The store type to operate on.
+ ///
/// The span to allocate the output of the operation.
///
- GarnetStatus PTTL(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output);
+ GarnetStatus TTL(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
#endregion
@@ -1263,19 +1268,10 @@ public interface IGarnetReadApi
/// Returns the absolute Unix timestamp (since January 1, 1970) in seconds at which the given key will expire.
///
/// The key to get the expiration time for.
- /// The type of store to retrieve the key from.
- /// The output containing the expiration time.
- /// The status of the operation.
- GarnetStatus EXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output);
-
- ///
- /// Returns the absolute Unix timestamp (since January 1, 1970) in milliseconds at which the given key will expire.
- ///
- /// The key to get the expiration time for.
- /// The type of store to retrieve the key from.
+ ///
/// The output containing the expiration time.
/// The status of the operation.
- GarnetStatus PEXPIRETIME(PinnedSpanByte key, StoreType storeType, ref SpanByteAndMemory output);
+ GarnetStatus EXPIRETIME(PinnedSpanByte key, ref UnifiedStoreInput input, ref GarnetUnifiedStoreOutput output);
#endregion
@@ -1296,7 +1292,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SortedSetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Returns the specified range of elements in the sorted set stored at key.
@@ -1357,7 +1353,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SortedSetLengthByValue(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// ZRANK: Returns the rank of member in the sorted set, the scores in the sorted set are ordered from low to high
@@ -1521,7 +1517,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus ListLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Gets the specified elements of the list stored at key.
@@ -1560,7 +1556,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus SetLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// SMEMBERS key
@@ -1716,7 +1712,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus HashStrLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Returns the number of fields contained in the hash Key.
@@ -1725,7 +1721,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus HashLength(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Returns if field is an existing field in the hash stored at key.
@@ -1743,7 +1739,7 @@ public interface IGarnetReadApi
///
///
///
- GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out ObjectOutputHeader output);
+ GarnetStatus HashExists(PinnedSpanByte key, ref ObjectInput input, out OutputHeader output);
///
/// Returns count random fields from the hash value.
diff --git a/libs/server/Cluster/CheckpointMetadata.cs b/libs/server/Cluster/CheckpointMetadata.cs
index 912e10462c4..798e02106a8 100644
--- a/libs/server/Cluster/CheckpointMetadata.cs
+++ b/libs/server/Cluster/CheckpointMetadata.cs
@@ -13,12 +13,6 @@ public sealed class CheckpointMetadata
public long storeCheckpointCoveredAofAddress;
public string storePrimaryReplId;
- public long objectStoreVersion;
- public Guid objectStoreHlogToken;
- public Guid objectStoreIndexToken;
- public long objectCheckpointCoveredAofAddress;
- public string objectStorePrimaryReplId;
-
public CheckpointMetadata()
{
storeVersion = -1;
@@ -26,12 +20,6 @@ public CheckpointMetadata()
storeIndexToken = default;
storeCheckpointCoveredAofAddress = long.MaxValue;
storePrimaryReplId = null;
-
- objectStoreVersion = -1;
- objectStoreHlogToken = default;
- objectStoreIndexToken = default;
- objectCheckpointCoveredAofAddress = long.MaxValue;
- objectStorePrimaryReplId = null;
}
///
@@ -45,12 +33,7 @@ public override string ToString()
$"storeHlogToken={storeHlogToken}," +
$"storeIndexToken={storeIndexToken}," +
$"storeCheckpointCoveredAofAddress={storeCheckpointCoveredAofAddress}," +
- $"storePrimaryReplId={storePrimaryReplId ?? "(empty)"}," +
- $"objectStoreVersion={objectStoreVersion}," +
- $"objectStoreHlogToken={objectStoreHlogToken}," +
- $"objectStoreIndexToken={objectStoreIndexToken}," +
- $"objectCheckpointCoveredAofAddress={objectCheckpointCoveredAofAddress}," +
- $"objectStorePrimaryReplId={objectStorePrimaryReplId ?? "(empty)"}";
+ $"storePrimaryReplId={storePrimaryReplId ?? "(empty)"}";
}
}
}
\ No newline at end of file
diff --git a/libs/server/Cluster/IClusterProvider.cs b/libs/server/Cluster/IClusterProvider.cs
index 8f169b4ae31..2a11d483ff3 100644
--- a/libs/server/Cluster/IClusterProvider.cs
+++ b/libs/server/Cluster/IClusterProvider.cs
@@ -13,10 +13,13 @@
namespace Garnet.server
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
ObjectAllocator>>>;
///
diff --git a/libs/server/Cluster/StoreType.cs b/libs/server/Cluster/StoreType.cs
index b841ba34c59..1efa27fb52e 100644
--- a/libs/server/Cluster/StoreType.cs
+++ b/libs/server/Cluster/StoreType.cs
@@ -10,6 +10,11 @@ namespace Garnet.server
///
public enum StoreType : byte
{
+ ///
+ /// No store specified
+ ///
+ None = 0,
+
///
/// Main (raw string) store
///
diff --git a/libs/server/Custom/CustomObjectBase.cs b/libs/server/Custom/CustomObjectBase.cs
index 3dddf4f1af9..e7a86f49317 100644
--- a/libs/server/Custom/CustomObjectBase.cs
+++ b/libs/server/Custom/CustomObjectBase.cs
@@ -83,7 +83,7 @@ public sealed override bool Operate(ref ObjectInput input, ref GarnetObjectStore
if ((byte)input.header.type != this.type)
{
// Indicates an incorrect type of key
- output.OutputFlags |= ObjectStoreOutputFlags.WrongType;
+ output.OutputFlags |= OutputFlags.WrongType;
output.SpanByteAndMemory.Length = 0;
return true;
}
diff --git a/libs/server/Custom/CustomRespCommands.cs b/libs/server/Custom/CustomRespCommands.cs
index 43cd3f8b013..c4514718b43 100644
--- a/libs/server/Custom/CustomRespCommands.cs
+++ b/libs/server/Custom/CustomRespCommands.cs
@@ -53,11 +53,11 @@ private bool TryTransactionProc(byte id, CustomTransactionProcedure proc, int st
return true;
}
- public bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, ref MemoryResult output)
+ public bool RunTransactionProc(byte id, ref CustomProcedureInput procInput, ref MemoryResult output, bool isRecovering = false)
{
var proc = customCommandManagerSession
.GetCustomTransactionProcedure(id, this, txnManager, scratchBufferAllocator, out _);
- return txnManager.RunTransactionProc(id, ref procInput, proc, ref output);
+ return txnManager.RunTransactionProc(id, ref procInput, proc, ref output, isRecovering);
}
private void TryCustomProcedure(CustomProcedure proc, int startIdx = 0)
diff --git a/libs/server/Custom/CustomTransactionProcedure.cs b/libs/server/Custom/CustomTransactionProcedure.cs
index 3c7210bd742..13bf8037ba3 100644
--- a/libs/server/Custom/CustomTransactionProcedure.cs
+++ b/libs/server/Custom/CustomTransactionProcedure.cs
@@ -36,10 +36,11 @@ public abstract class CustomTransactionProcedure : CustomProcedureBase
///
///
///
- ///
- protected void AddKey(PinnedSpanByte key, LockType type, bool isObject)
+ ///
+ protected void AddKey(PinnedSpanByte key, LockType type, StoreType storeType)
{
- txnManager.SaveKeyEntryToLock(key, isObject, type);
+ txnManager.AddTransactionStoreType(storeType);
+ txnManager.SaveKeyEntryToLock(key, type);
txnManager.VerifyKeyOwnership(key, type);
}
@@ -76,6 +77,8 @@ public abstract void Main(TGarnetApi api, ref CustomProcedureInput p
///
/// Finalize transaction: runs after the transactions commits/aborts, allowed to read and write (non-transactionally) with per-key locks and produce output
+ /// NOTE: Finalize is considered post transaction processing and therefore is not executed at recovery time. Instead, the individual Tsavorite commands are logged and replayed through the AOF.
+ /// If you are not using AOF for persistence then this is implementation detail you can ignore.
///
public virtual void Finalize(TGarnetApi api, ref CustomProcedureInput procInput, ref MemoryResult output)
where TGarnetApi : IGarnetApi
diff --git a/libs/server/Databases/DatabaseManagerBase.cs b/libs/server/Databases/DatabaseManagerBase.cs
index a889863afdd..f06be9d602b 100644
--- a/libs/server/Databases/DatabaseManagerBase.cs
+++ b/libs/server/Databases/DatabaseManagerBase.cs
@@ -10,11 +10,8 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Base class for logical database management
@@ -40,8 +37,7 @@ internal abstract class DatabaseManagerBase : IDatabaseManager
public abstract void ResumeCheckpoints(int dbId);
///
- public abstract void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false,
- bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null);
+ public abstract void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null);
///
public abstract bool TakeCheckpoint(bool background, ILogger logger = null, CancellationToken token = default);
@@ -84,7 +80,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit,
public abstract void ExpiredKeyDeletionScan();
///
- public abstract void StartObjectSizeTrackers(CancellationToken token = default);
+ public abstract void StartSizeTrackers(CancellationToken token = default);
///
public abstract void Reset(int dbId = 0);
@@ -120,10 +116,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit,
public abstract IDatabaseManager Clone(bool enableAof);
///
- public TsavoriteKV MainStore => DefaultDatabase.MainStore;
-
- ///
- public TsavoriteKV ObjectStore => DefaultDatabase.ObjectStore;
+ public TsavoriteKV Store => DefaultDatabase.Store;
///
public TsavoriteLog AppendOnlyFile => DefaultDatabase.AppendOnlyFile;
@@ -132,7 +125,7 @@ public abstract Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit,
public DateTimeOffset LastSaveTime => DefaultDatabase.LastSaveTime;
///
- public CacheSizeTracker ObjectStoreSizeTracker => DefaultDatabase.ObjectStoreSizeTracker;
+ public CacheSizeTracker SizeTracker => DefaultDatabase.SizeTracker;
///
public WatchVersionMap VersionMap => DefaultDatabase.VersionMap;
@@ -174,37 +167,14 @@ protected DatabaseManagerBase(StoreWrapper.DatabaseCreatorDelegate createDatabas
///
/// Database to recover
/// Store version
- /// Object store version
- protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersion, out long objectStoreVersion)
+ protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersion)
{
storeVersion = 0;
- objectStoreVersion = 0;
- if (db.ObjectStore != null)
- {
- // Get store recover version
- var currStoreVersion = db.MainStore.GetRecoverVersion();
- // Get object store recover version
- var currObjectStoreVersion = db.ObjectStore.GetRecoverVersion();
-
- // Choose the minimum common recover version for both stores
- if (currStoreVersion < currObjectStoreVersion)
- currObjectStoreVersion = currStoreVersion;
- else if (objectStoreVersion > 0) // handle the case where object store was disabled at checkpointing time
- currStoreVersion = currObjectStoreVersion;
-
- // Recover to the minimum common recover version
- storeVersion = db.MainStore.Recover(recoverTo: currStoreVersion);
- objectStoreVersion = db.ObjectStore.Recover(recoverTo: currObjectStoreVersion);
- Logger?.LogInformation("Recovered store to version {storeVersion} and object store to version {objectStoreVersion}", storeVersion, objectStoreVersion);
- }
- else
- {
- storeVersion = db.MainStore.Recover();
- Logger?.LogInformation("Recovered store to version {storeVersion}", storeVersion);
- }
+ storeVersion = db.Store.Recover();
+ Logger?.LogInformation("Recovered store to version {storeVersion}", storeVersion);
- if (storeVersion > 0 || objectStoreVersion > 0)
+ if (storeVersion > 0)
{
db.LastSaveTime = DateTimeOffset.UtcNow;
}
@@ -217,36 +187,31 @@ protected void RecoverDatabaseCheckpoint(GarnetDatabase db, out long storeVersio
/// Logger
/// Cancellation token
/// Tuple of store tail address and object store tail address
- protected async Task<(long?, long?)> TakeCheckpointAsync(GarnetDatabase db, ILogger logger = null, CancellationToken token = default)
+ protected async Task TakeCheckpointAsync(GarnetDatabase db, ILogger logger = null, CancellationToken token = default)
{
try
{
DoCompaction(db, isFromCheckpoint: true, logger);
- var lastSaveStoreTailAddress = db.MainStore.Log.TailAddress;
- var lastSaveObjectStoreTailAddress = (db.ObjectStore?.Log.TailAddress).GetValueOrDefault();
+ var lastSaveStoreTailAddress = db.Store.Log.TailAddress;
var full = db.LastSaveStoreTailAddress == 0 ||
- lastSaveStoreTailAddress - db.LastSaveStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval ||
- (db.ObjectStore != null && (db.LastSaveObjectStoreTailAddress == 0 ||
- lastSaveObjectStoreTailAddress - db.LastSaveObjectStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval));
+ lastSaveStoreTailAddress - db.LastSaveStoreTailAddress >= StoreWrapper.serverOptions.FullCheckpointLogInterval;
var tryIncremental = StoreWrapper.serverOptions.EnableIncrementalSnapshots;
- if (db.MainStore.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit)
- tryIncremental = false;
- if (db.ObjectStore?.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit)
+ if (db.Store.IncrementalSnapshotTailAddress >= StoreWrapper.serverOptions.IncrementalSnapshotLogSizeLimit)
tryIncremental = false;
var checkpointType = StoreWrapper.serverOptions.UseFoldOverCheckpoints ? CheckpointType.FoldOver : CheckpointType.Snapshot;
await InitiateCheckpointAsync(db, full, checkpointType, tryIncremental, logger);
- return full ? new(lastSaveStoreTailAddress, lastSaveObjectStoreTailAddress) : (null, null);
+ return full ? lastSaveStoreTailAddress : null;
}
catch (Exception ex)
{
logger?.LogError(ex, "Checkpointing threw exception, DB ID: {id}", db.Id);
}
- return (null, null);
+ return null;
}
///
@@ -310,10 +275,8 @@ protected void ResetDatabase(GarnetDatabase db)
{
try
{
- if (db.MainStore.Log.TailAddress > 64)
- db.MainStore.Reset();
- if (db.ObjectStore?.Log.TailAddress > 64)
- db.ObjectStore?.Reset();
+ if (db.Store.Log.TailAddress > 64)
+ db.Store.Reset();
db.AppendOnlyFile?.Reset();
var lastSave = DateTimeOffset.FromUnixTimeSeconds(0);
@@ -353,8 +316,7 @@ protected static void EnqueueDatabaseCommit(GarnetDatabase db, AofEntryType entr
/// Truncate AOF log
protected static void FlushDatabase(GarnetDatabase db, bool unsafeTruncateLog, bool truncateAof = true)
{
- db.MainStore.Log.ShiftBeginAddress(db.MainStore.Log.TailAddress, truncateLog: unsafeTruncateLog);
- db.ObjectStore?.Log.ShiftBeginAddress(db.ObjectStore.Log.TailAddress, truncateLog: unsafeTruncateLog);
+ db.Store.Log.ShiftBeginAddress(db.Store.Log.TailAddress, truncateLog: unsafeTruncateLog);
if (truncateAof)
db.AppendOnlyFile?.TruncateUntil(db.AppendOnlyFile.TailAddress);
@@ -369,30 +331,13 @@ protected bool GrowIndexesIfNeeded(GarnetDatabase db)
{
var indexesMaxedOut = true;
- if (!DefaultDatabase.MainStoreIndexMaxedOut)
+ if (!DefaultDatabase.StoreIndexMaxedOut)
{
- var dbMainStore = DefaultDatabase.MainStore;
- if (GrowIndexIfNeeded(StoreType.Main,
- StoreWrapper.serverOptions.AdjustedIndexMaxCacheLines, dbMainStore.OverflowBucketAllocations,
+ var dbMainStore = DefaultDatabase.Store;
+ if (GrowIndexIfNeeded(StoreWrapper.serverOptions.AdjustedIndexMaxCacheLines, dbMainStore.OverflowBucketAllocations,
() => dbMainStore.IndexSize, async () => await dbMainStore.GrowIndexAsync()))
{
- db.MainStoreIndexMaxedOut = true;
- }
- else
- {
- indexesMaxedOut = false;
- }
- }
-
- if (!db.ObjectStoreIndexMaxedOut)
- {
- var dbObjectStore = db.ObjectStore;
- if (GrowIndexIfNeeded(StoreType.Object,
- StoreWrapper.serverOptions.AdjustedObjectStoreIndexMaxCacheLines,
- dbObjectStore.OverflowBucketAllocations,
- () => dbObjectStore.IndexSize, async () => await dbObjectStore.GrowIndexAsync()))
- {
- db.ObjectStoreIndexMaxedOut = true;
+ db.StoreIndexMaxedOut = true;
}
else
{
@@ -410,15 +355,15 @@ protected bool GrowIndexesIfNeeded(GarnetDatabase db)
/// Logger
protected void ExecuteObjectCollection(GarnetDatabase db, ILogger logger = null)
{
- if (db.ObjectStoreCollectionDbStorageSession == null)
+ if (db.StoreCollectionDbStorageSession == null)
{
var scratchBufferManager = new ScratchBufferBuilder();
- db.ObjectStoreCollectionDbStorageSession =
+ db.StoreCollectionDbStorageSession =
new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
}
- ExecuteHashCollect(db.ObjectStoreCollectionDbStorageSession);
- ExecuteSortedSetCollect(db.ObjectStoreCollectionDbStorageSession);
+ ExecuteHashCollect(db.StoreCollectionDbStorageSession);
+ ExecuteSortedSetCollect(db.StoreCollectionDbStorageSession);
}
///
@@ -427,12 +372,7 @@ protected void ExecuteObjectCollection(GarnetDatabase db, ILogger logger = null)
/// Database
protected void ExpiredKeyDeletionScan(GarnetDatabase db)
{
- _ = MainStoreExpiredKeyDeletionScan(db);
-
- if (StoreWrapper.serverOptions.DisableObjects)
- return;
-
- _ = ObjectStoreExpiredKeyDeletionScan(db);
+ _ = StoreExpiredKeyDeletionScan(db);
}
///
@@ -448,8 +388,7 @@ protected void DoCompaction(GarnetDatabase db, bool isFromCheckpoint = false, IL
// If periodic compaction is enabled and this is called from checkpointing, skip compaction
if (isFromCheckpoint && StoreWrapper.serverOptions.CompactionFrequencySecs > 0) return;
- DoCompaction(db, StoreWrapper.serverOptions.CompactionMaxSegments,
- StoreWrapper.serverOptions.ObjectStoreCompactionMaxSegments, 1,
+ DoCompaction(db, StoreWrapper.serverOptions.CompactionMaxSegments, 1,
StoreWrapper.serverOptions.CompactionType, StoreWrapper.serverOptions.CompactionForceDelete);
}
catch (Exception ex)
@@ -466,125 +405,79 @@ protected void DoCompaction(GarnetDatabase db, bool isFromCheckpoint = false, IL
/// Decision is based on whether overflow bucket allocation is more than a threshold which indicates a contention
/// in the index leading many allocations to the same bucket.
///
- ///
///
///
///
///
/// True if index has reached its max size
- protected bool GrowIndexIfNeeded(StoreType storeType, long indexMaxSize, long overflowCount, Func indexSizeRetriever, Action growAction)
+ protected bool GrowIndexIfNeeded(long indexMaxSize, long overflowCount, Func indexSizeRetriever, Action growAction)
{
Logger?.LogDebug(
- $"IndexAutoGrowTask[{{storeType}}]: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}",
- storeType, indexSizeRetriever(), indexMaxSize, overflowCount);
+ $"IndexAutoGrowTask: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}",
+ indexSizeRetriever(), indexMaxSize, overflowCount);
if (indexSizeRetriever() < indexMaxSize &&
overflowCount > (indexSizeRetriever() * StoreWrapper.serverOptions.IndexResizeThreshold / 100))
{
Logger?.LogInformation(
- $"IndexAutoGrowTask[{{storeType}}]: overflowCount {{overflowCount}} ratio more than threshold {{indexResizeThreshold}}%. Doubling index size...",
- storeType, overflowCount, StoreWrapper.serverOptions.IndexResizeThreshold);
+ $"IndexAutoGrowTask: overflowCount {{overflowCount}} ratio more than threshold {{indexResizeThreshold}}%. Doubling index size...",
+ overflowCount, StoreWrapper.serverOptions.IndexResizeThreshold);
growAction();
}
if (indexSizeRetriever() < indexMaxSize) return false;
Logger?.LogDebug(
- $"IndexAutoGrowTask[{{storeType}}]: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}",
- storeType, indexSizeRetriever(), indexMaxSize, overflowCount);
+ $"IndexAutoGrowTask: checking index size {{indexSizeRetriever}} against max {{indexMaxSize}} with overflow {{overflowCount}}",
+ indexSizeRetriever(), indexMaxSize, overflowCount);
return true;
}
- private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int objectStoreMaxSegments, int numSegmentsToCompact, LogCompactionType compactionType, bool compactionForceDelete)
+ private void DoCompaction(GarnetDatabase db, int mainStoreMaxSegments, int numSegmentsToCompact, LogCompactionType compactionType, bool compactionForceDelete)
{
if (compactionType == LogCompactionType.None) return;
- var mainStoreLog = db.MainStore.Log;
+ var storeLog = db.Store.Log;
var mainStoreMaxLogSize = (1L << StoreWrapper.serverOptions.SegmentSizeBits()) * mainStoreMaxSegments;
- if (mainStoreLog.ReadOnlyAddress - mainStoreLog.BeginAddress > mainStoreMaxLogSize)
+ if (storeLog.ReadOnlyAddress - storeLog.BeginAddress > mainStoreMaxLogSize)
{
- var readOnlyAddress = mainStoreLog.ReadOnlyAddress;
+ var readOnlyAddress = storeLog.ReadOnlyAddress;
var compactLength = (1L << StoreWrapper.serverOptions.SegmentSizeBits()) * (mainStoreMaxSegments - numSegmentsToCompact);
var untilAddress = readOnlyAddress - compactLength;
Logger?.LogInformation(
"Begin main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}",
- untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress);
+ untilAddress, storeLog.BeginAddress, readOnlyAddress, storeLog.TailAddress);
switch (compactionType)
{
case LogCompactionType.Shift:
- mainStoreLog.ShiftBeginAddress(untilAddress, true, compactionForceDelete);
+ storeLog.ShiftBeginAddress(untilAddress, true, compactionForceDelete);
break;
case LogCompactionType.Scan:
- mainStoreLog.Compact(untilAddress, CompactionType.Scan);
+ storeLog.Compact(untilAddress, CompactionType.Scan);
if (compactionForceDelete)
{
CompactionCommitAof(db);
- mainStoreLog.Truncate();
+ storeLog.Truncate();
}
break;
case LogCompactionType.Lookup:
- mainStoreLog.Compact(untilAddress, CompactionType.Lookup);
+ storeLog.Compact(untilAddress, CompactionType.Lookup);
if (compactionForceDelete)
{
CompactionCommitAof(db);
- mainStoreLog.Truncate();
+ storeLog.Truncate();
}
break;
}
Logger?.LogInformation(
- "End main store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}",
- untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress);
- }
-
- if (db.ObjectStore == null) return;
-
- var objectStoreLog = db.ObjectStore.Log;
-
- var objectStoreMaxLogSize = (1L << StoreWrapper.serverOptions.ObjectStoreSegmentSizeBits()) * objectStoreMaxSegments;
-
- if (objectStoreLog.ReadOnlyAddress - objectStoreLog.BeginAddress > objectStoreMaxLogSize)
- {
- var readOnlyAddress = objectStoreLog.ReadOnlyAddress;
- var compactLength = (1L << StoreWrapper.serverOptions.ObjectStoreSegmentSizeBits()) * (objectStoreMaxSegments - numSegmentsToCompact);
- var untilAddress = readOnlyAddress - compactLength;
- Logger?.LogInformation(
- "Begin object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}",
- untilAddress, objectStoreLog.BeginAddress, readOnlyAddress, objectStoreLog.TailAddress);
-
- switch (compactionType)
- {
- case LogCompactionType.Shift:
- objectStoreLog.ShiftBeginAddress(untilAddress, compactionForceDelete);
- break;
-
- case LogCompactionType.Scan:
- objectStoreLog.Compact(untilAddress, CompactionType.Scan);
- if (compactionForceDelete)
- {
- CompactionCommitAof(db);
- objectStoreLog.Truncate();
- }
- break;
-
- case LogCompactionType.Lookup:
- objectStoreLog.Compact(untilAddress, CompactionType.Lookup);
- if (compactionForceDelete)
- {
- CompactionCommitAof(db);
- objectStoreLog.Truncate();
- }
- break;
- }
-
- Logger?.LogInformation(
- "End object store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}",
- untilAddress, mainStoreLog.BeginAddress, readOnlyAddress, mainStoreLog.TailAddress);
+ "End store compact until {untilAddress}, Begin = {beginAddress}, ReadOnly = {readOnlyAddress}, Tail = {tailAddress}",
+ untilAddress, storeLog.BeginAddress, readOnlyAddress, storeLog.TailAddress);
}
}
@@ -642,28 +535,15 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo
IStateMachine sm;
if (full)
{
- sm = db.ObjectStore == null ?
- Checkpoint.Full(db.MainStore, checkpointType, out checkpointResult.token) :
- Checkpoint.Full(db.MainStore, db.ObjectStore, checkpointType, out checkpointResult.token);
+ sm = Checkpoint.Full(db.Store, checkpointType, out checkpointResult.token);
}
else
{
- tryIncremental = tryIncremental && db.MainStore.CanTakeIncrementalCheckpoint(checkpointType, out checkpointResult.token);
- if (db.ObjectStore != null)
- tryIncremental = tryIncremental && db.ObjectStore.CanTakeIncrementalCheckpoint(checkpointType, out var guid2) && checkpointResult.token == guid2;
+ tryIncremental = tryIncremental && db.Store.CanTakeIncrementalCheckpoint(checkpointType, out checkpointResult.token);
- if (tryIncremental)
- {
- sm = db.ObjectStore == null ?
- Checkpoint.IncrementalHybridLogOnly(db.MainStore, checkpointResult.token) :
- Checkpoint.IncrementalHybridLogOnly(db.MainStore, db.ObjectStore, checkpointResult.token);
- }
- else
- {
- sm = db.ObjectStore == null ?
- Checkpoint.HybridLogOnly(db.MainStore, checkpointType, out checkpointResult.token) :
- Checkpoint.HybridLogOnly(db.MainStore, db.ObjectStore, checkpointType, out checkpointResult.token);
- }
+ sm = tryIncremental
+ ? Checkpoint.IncrementalHybridLogOnly(db.Store, checkpointResult.token)
+ : Checkpoint.HybridLogOnly(db.Store, checkpointType, out checkpointResult.token);
}
checkpointResult.success = await db.StateMachineDriver.RunAsync(sm);
@@ -680,18 +560,15 @@ private async Task InitiateCheckpointAsync(GarnetDatabase db, bool full, Checkpo
db.AppendOnlyFile?.Commit();
}
- if (db.ObjectStore != null)
+ // During the checkpoint, we may have serialized Garnet objects in (v) versions of objects.
+ // We can now safely remove these serialized versions as they are no longer needed.
+ using var iter1 = db.Store.Log.Scan(db.Store.Log.ReadOnlyAddress,
+ db.Store.Log.TailAddress, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: true);
+ while (iter1.GetNext())
{
- // During the checkpoint, we may have serialized Garnet objects in (v) versions of objects.
- // We can now safely remove these serialized versions as they are no longer needed.
- using var iter1 = db.ObjectStore.Log.Scan(db.ObjectStore.Log.ReadOnlyAddress,
- db.ObjectStore.Log.TailAddress, DiskScanBufferingMode.SinglePageBuffering, includeClosedRecords: true);
- while (iter1.GetNext())
- {
- var valueObject = iter1.ValueObject;
- if (valueObject != null)
- ((GarnetObjectBase)iter1.ValueObject).ClearSerializedObjectData();
- }
+ var valueObject = iter1.ValueObject;
+ if (valueObject != null)
+ ((GarnetObjectBase)iter1.ValueObject).ClearSerializedObjectData();
}
logger?.LogInformation("Completed checkpoint for DB ID: {id}", db.Id);
@@ -716,34 +593,20 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession)
///
public abstract (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId);
- protected (long numExpiredKeysFound, long totalRecordsScanned) MainStoreExpiredKeyDeletionScan(GarnetDatabase db)
+ protected (long numExpiredKeysFound, long totalRecordsScanned) StoreExpiredKeyDeletionScan(GarnetDatabase db)
{
- if (db.MainStoreExpiredKeyDeletionDbStorageSession == null)
+ if (db.StoreExpiredKeyDeletionDbStorageSession == null)
{
var scratchBufferManager = new ScratchBufferBuilder();
- db.MainStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
+ db.StoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
}
var scanFrom = StoreWrapper.store.Log.ReadOnlyAddress;
var scanUntil = StoreWrapper.store.Log.TailAddress;
- (var deletedCount, var totalCount) = db.MainStoreExpiredKeyDeletionDbStorageSession.MainStoreExpiredKeyDeletionScan(scanFrom, scanUntil);
- Logger?.LogDebug("Main Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}", deletedCount, totalCount, scanFrom, scanUntil, db.Id);
-
- return (deletedCount, totalCount);
- }
-
- protected (long numExpiredKeysFound, long totalRecordsScanned) ObjectStoreExpiredKeyDeletionScan(GarnetDatabase db)
- {
- if (db.ObjectStoreExpiredKeyDeletionDbStorageSession == null)
- {
- var scratchBufferManager = new ScratchBufferBuilder();
- db.ObjectStoreExpiredKeyDeletionDbStorageSession = new StorageSession(StoreWrapper, scratchBufferManager, null, null, db.Id, Logger);
- }
-
- var scanFrom = StoreWrapper.objectStore.Log.ReadOnlyAddress;
- var scanUntil = StoreWrapper.objectStore.Log.TailAddress;
- (var deletedCount, var totalCount) = db.ObjectStoreExpiredKeyDeletionDbStorageSession.ObjectStoreExpiredKeyDeletionScan(scanFrom, scanUntil);
- Logger?.LogDebug("Object Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}", deletedCount, totalCount, scanFrom, scanUntil, db.Id);
+ var (deletedCount, totalCount) = db.StoreExpiredKeyDeletionDbStorageSession.ExpiredKeyDeletionScan(scanFrom, scanUntil);
+ Logger?.LogDebug(
+ "Store - Deleted {deletedCount} keys out {totalCount} records in range {scanFrom} to {scanUntil} for DB {id}",
+ deletedCount, totalCount, scanFrom, scanUntil, db.Id);
return (deletedCount, totalCount);
}
@@ -753,16 +616,12 @@ private static void ExecuteSortedSetCollect(StorageSession storageSession)
protected (HybridLogScanMetrics mainStore, HybridLogScanMetrics objectStore) CollectHybridLogStatsForDb(GarnetDatabase db)
{
- FunctionsState functionsState = CreateFunctionsState();
- MainSessionFunctions mainStoreSessionFuncs = new MainSessionFunctions(functionsState);
- var mainStoreStats = CollectHybridLogStats(db, db.MainStore, mainStoreSessionFuncs);
+ var functionsState = CreateFunctionsState();
+ var mainStoreSessionFunctions = new MainSessionFunctions(functionsState);
+ var mainStoreStats = CollectHybridLogStats(db, db.Store, mainStoreSessionFunctions);
- HybridLogScanMetrics objectStoreStats = null;
- if (ObjectStore != null)
- {
- ObjectSessionFunctions objectSessionFunctions = new ObjectSessionFunctions(functionsState);
- objectStoreStats = CollectHybridLogStats(db, db.ObjectStore, objectSessionFunctions);
- }
+ var objectSessionFunctions = new ObjectSessionFunctions(functionsState);
+ var objectStoreStats = CollectHybridLogStats(db, db.Store, objectSessionFunctions);
return (mainStoreStats, objectStoreStats);
}
@@ -791,7 +650,7 @@ private HybridLogScanMetrics CollectHybridLogStats= db.MainStore.Log.ReadOnlyAddress ? "Mutable" : "Immutable";
+ string region = iter.CurrentAddress >= db.Store.Log.ReadOnlyAddress ? "Mutable" : "Immutable";
string state = "Live";
if (iter.Info.IsSealed)
{
diff --git a/libs/server/Databases/DatabaseManagerFactory.cs b/libs/server/Databases/DatabaseManagerFactory.cs
index a1a9bae89a8..12de2907e44 100644
--- a/libs/server/Databases/DatabaseManagerFactory.cs
+++ b/libs/server/Databases/DatabaseManagerFactory.cs
@@ -37,7 +37,7 @@ private static bool ShouldCreateMultipleDatabaseManager(GarnetServerOptions serv
using (createDatabaseDelegate(0))
{
// Check if there are multiple databases to recover from checkpoint
- var checkpointParentDir = serverOptions.MainStoreCheckpointBaseDirectory;
+ var checkpointParentDir = serverOptions.StoreCheckpointBaseDirectory;
var checkpointDirBaseName = serverOptions.GetCheckpointDirectoryName(0);
if (MultiDatabaseManager.TryGetSavedDatabaseIds(checkpointParentDir, checkpointDirBaseName,
diff --git a/libs/server/Databases/IDatabaseManager.cs b/libs/server/Databases/IDatabaseManager.cs
index a49ddc6a494..709db854b1c 100644
--- a/libs/server/Databases/IDatabaseManager.cs
+++ b/libs/server/Databases/IDatabaseManager.cs
@@ -10,11 +10,8 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Interface for logical database management
@@ -29,12 +26,7 @@ public interface IDatabaseManager : IDisposable
///
/// Store (of DB 0)
///
- public TsavoriteKV MainStore { get; }
-
- ///
- /// Object store (of DB 0)
- ///
- public TsavoriteKV ObjectStore { get; }
+ public TsavoriteKV Store { get; }
///
/// AOF (of DB 0)
@@ -49,7 +41,7 @@ public interface IDatabaseManager : IDisposable
///
/// Object store size tracker (of DB 0)
///
- public CacheSizeTracker ObjectStoreSizeTracker { get; }
+ public CacheSizeTracker SizeTracker { get; }
///
/// Version map (of DB 0)
@@ -93,9 +85,8 @@ public interface IDatabaseManager : IDisposable
///
///
///
- ///
///
- public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null);
+ public void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null);
///
/// Take checkpoint of all active databases if checkpointing is not in progress
@@ -191,9 +182,9 @@ public Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLimit, Cancellati
public void ExpiredKeyDeletionScan();
///
- /// Start object size trackers for all active databases
+ /// Start size trackers for all active databases
///
- public void StartObjectSizeTrackers(CancellationToken token = default);
+ public void StartSizeTrackers(CancellationToken token = default);
///
/// Reset
diff --git a/libs/server/Databases/MultiDatabaseManager.cs b/libs/server/Databases/MultiDatabaseManager.cs
index b5df1be96fb..9759e4a4dc6 100644
--- a/libs/server/Databases/MultiDatabaseManager.cs
+++ b/libs/server/Databases/MultiDatabaseManager.cs
@@ -87,13 +87,13 @@ public MultiDatabaseManager(SingleDatabaseManager src) :
}
///
- public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null)
+ public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null)
{
if (replicaRecover)
throw new GarnetException(
$"Unexpected call to {nameof(MultiDatabaseManager)}.{nameof(RecoverCheckpoint)} with {nameof(replicaRecover)} == true.");
- var checkpointParentDir = StoreWrapper.serverOptions.MainStoreCheckpointBaseDirectory;
+ var checkpointParentDir = StoreWrapper.serverOptions.StoreCheckpointBaseDirectory;
var checkpointDirBaseName = StoreWrapper.serverOptions.GetCheckpointDirectoryName(0);
int[] dbIdsToRecover;
@@ -122,7 +122,7 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover
try
{
- RecoverDatabaseCheckpoint(db, out storeVersion, out objectStoreVersion);
+ RecoverDatabaseCheckpoint(db, out storeVersion);
}
catch (TsavoriteNoHybridLogException ex)
{
@@ -139,14 +139,6 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover
if (StoreWrapper.serverOptions.FailOnRecoveryError)
throw;
}
-
- // After recovery, we check if store versions match
- if (db.ObjectStore != null && storeVersion != objectStoreVersion)
- {
- Logger?.LogInformation("Main store and object store checkpoint versions do not match; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion);
- if (StoreWrapper.serverOptions.FailOnRecoveryError)
- throw new GarnetException("Main store and object store checkpoint versions do not match");
- }
}
}
@@ -210,9 +202,8 @@ public override bool TakeCheckpoint(bool background, int dbId, ILogger logger =
{
if (t.IsCompletedSuccessfully)
{
- var storeTailAddress = t.Result.Item1;
- var objectStoreTailAddress = t.Result.Item2;
- UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress);
+ var storeTailAddress = t.Result;
+ UpdateLastSaveData(dbId, storeTailAddress);
}
}
finally
@@ -249,9 +240,8 @@ public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime,
// Necessary to take a checkpoint because the latest checkpoint is before entryTime
var result = await TakeCheckpointAsync(db, logger: Logger);
- var storeTailAddress = result.Item1;
- var objectStoreTailAddress = result.Item2;
- UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress);
+ var storeTailAddress = result;
+ UpdateLastSaveData(dbId, storeTailAddress);
}
finally
{
@@ -575,7 +565,7 @@ public override void ExpiredKeyDeletionScan()
}
///
- public override void StartObjectSizeTrackers(CancellationToken token = default)
+ public override void StartSizeTrackers(CancellationToken token = default)
{
sizeTrackersStarted = true;
@@ -595,7 +585,7 @@ public override void StartObjectSizeTrackers(CancellationToken token = default)
var db = databasesMapSnapshot[dbId];
Debug.Assert(db != null);
- db.ObjectStoreSizeTracker?.Start(token);
+ db.SizeTracker?.Start(token);
}
}
finally
@@ -624,8 +614,7 @@ public override void ResetRevivificationStats()
for (var i = 0; i < activeDbIdsMapSize; i++)
{
var dbId = activeDbIdsMapSnapshot[i];
- databaseMapSnapshot[dbId].MainStore.ResetRevivificationStats();
- databaseMapSnapshot[dbId].ObjectStore?.ResetRevivificationStats();
+ databaseMapSnapshot[dbId].Store.ResetRevivificationStats();
}
}
@@ -711,7 +700,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto
if (!success)
throw new GarnetException($"Database with ID {dbId} was not found.");
- return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper, memoryPool: null, db.ObjectStoreSizeTracker, Logger, respProtocolVersion);
+ return new(db.AppendOnlyFile, db.VersionMap, StoreWrapper, memoryPool: null, db.SizeTracker, Logger, respProtocolVersion);
}
///
@@ -931,7 +920,7 @@ private void HandleDatabaseAdded(int dbId)
// If size tracker exists and is stopped, start it (only if DB 0 size tracker is started as well)
var db = databases.Map[dbId];
if (sizeTrackersStarted)
- db.ObjectStoreSizeTracker?.Start(StoreWrapper.ctsCommit.Token);
+ db.SizeTracker?.Start(StoreWrapper.ctsCommit.Token);
activeDbIds.TryGetNextId(out var nextIdx);
activeDbIds.TrySetValue(nextIdx, db.Id);
@@ -1018,9 +1007,8 @@ private async Task TakeDatabasesCheckpointAsync(int dbIdsCount, ILogger lo
if (!t.IsCompletedSuccessfully)
return;
- var storeTailAddress = t.Result.Item1;
- var objectStoreTailAddress = t.Result.Item2;
- UpdateLastSaveData(dbId, storeTailAddress, objectStoreTailAddress);
+ var storeTailAddress = t.Result;
+ UpdateLastSaveData(dbId, storeTailAddress);
}, TaskContinuationOptions.ExecuteSynchronously);
}
@@ -1038,7 +1026,7 @@ private async Task TakeDatabasesCheckpointAsync(int dbIdsCount, ILogger lo
return true;
}
- private void UpdateLastSaveData(int dbId, long? storeTailAddress, long? objectStoreTailAddress)
+ private void UpdateLastSaveData(int dbId, long? storeTailAddress)
{
var databasesMapSnapshot = databases.Map;
@@ -1048,9 +1036,6 @@ private void UpdateLastSaveData(int dbId, long? storeTailAddress, long? objectSt
if (storeTailAddress.HasValue)
{
db.LastSaveStoreTailAddress = storeTailAddress.Value;
-
- if (db.ObjectStore != null && objectStoreTailAddress.HasValue)
- db.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value;
}
}
@@ -1070,11 +1055,7 @@ public override void Dispose()
}
public override (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId)
- {
- var (k1, t1) = MainStoreExpiredKeyDeletionScan(GetDbById(dbId));
- var (k2, t2) = StoreWrapper.serverOptions.DisableObjects ? (0, 0) : ObjectStoreExpiredKeyDeletionScan(GetDbById(dbId));
- return (k1 + k2, t1 + t2);
- }
+ => StoreExpiredKeyDeletionScan(GetDbById(dbId));
private GarnetDatabase GetDbById(int dbId)
{
diff --git a/libs/server/Databases/SingleDatabaseManager.cs b/libs/server/Databases/SingleDatabaseManager.cs
index 0a4b6053713..a7570118a2c 100644
--- a/libs/server/Databases/SingleDatabaseManager.cs
+++ b/libs/server/Databases/SingleDatabaseManager.cs
@@ -4,7 +4,6 @@
using System;
using System.Threading;
using System.Threading.Tasks;
-using Garnet.common;
using Garnet.server.Metrics;
using Microsoft.Extensions.Logging;
using Tsavorite.core;
@@ -54,9 +53,9 @@ public override GarnetDatabase TryGetOrAddDatabase(int dbId, out bool success, o
}
///
- public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, bool recoverObjectStoreFromToken = false, CheckpointMetadata metadata = null)
+ public override void RecoverCheckpoint(bool replicaRecover = false, bool recoverMainStoreFromToken = false, CheckpointMetadata metadata = null)
{
- long storeVersion = 0, objectStoreVersion = 0;
+ long storeVersion = 0;
try
{
if (replicaRecover)
@@ -64,49 +63,33 @@ public override void RecoverCheckpoint(bool replicaRecover = false, bool recover
// Note: Since replicaRecover only pertains to cluster-mode, we can use the default store pointers (since multi-db mode is disabled in cluster-mode)
if (metadata!.storeIndexToken != default && metadata.storeHlogToken != default)
{
- storeVersion = !recoverMainStoreFromToken ? MainStore.Recover() : MainStore.Recover(metadata.storeIndexToken, metadata.storeHlogToken);
+ storeVersion = !recoverMainStoreFromToken ? Store.Recover() : Store.Recover(metadata.storeIndexToken, metadata.storeHlogToken);
}
- if (ObjectStore != null)
- {
- if (metadata.objectStoreIndexToken != default && metadata.objectStoreHlogToken != default)
- {
- objectStoreVersion = !recoverObjectStoreFromToken ? ObjectStore.Recover() : ObjectStore.Recover(metadata.objectStoreIndexToken, metadata.objectStoreHlogToken);
- }
- }
-
- if (storeVersion > 0 || objectStoreVersion > 0)
+ if (storeVersion > 0)
defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow;
}
else
{
- RecoverDatabaseCheckpoint(defaultDatabase, out storeVersion, out objectStoreVersion);
+ RecoverDatabaseCheckpoint(defaultDatabase, out storeVersion);
}
}
catch (TsavoriteNoHybridLogException ex)
{
// No hybrid log being found is not the same as an error in recovery. e.g. fresh start
Logger?.LogInformation(ex,
- "No Hybrid Log found for recovery; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}",
- storeVersion, objectStoreVersion);
+ "No Hybrid Log found for recovery; storeVersion = {storeVersion};",
+ storeVersion);
}
catch (Exception ex)
{
Logger?.LogInformation(ex,
- "Error during recovery of store; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}",
- storeVersion, objectStoreVersion);
+ "Error during recovery of store; storeVersion = {storeVersion};",
+ storeVersion);
if (StoreWrapper.serverOptions.FailOnRecoveryError)
throw;
}
-
- // After recovery, we check if store versions match
- if (ObjectStore != null && storeVersion != objectStoreVersion)
- {
- Logger?.LogInformation("Main store and object store checkpoint versions do not match; storeVersion = {storeVersion}; objectStoreVersion = {objectStoreVersion}", storeVersion, objectStoreVersion);
- if (StoreWrapper.serverOptions.FailOnRecoveryError)
- throw new GarnetException("Main store and object store checkpoint versions do not match");
- }
}
///
@@ -139,13 +122,10 @@ public override bool TakeCheckpoint(bool background, ILogger logger = null, Canc
{
if (t.IsCompletedSuccessfully)
{
- var storeTailAddress = t.Result.Item1;
- var objectStoreTailAddress = t.Result.Item2;
+ var storeTailAddress = t.Result;
if (storeTailAddress.HasValue)
defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value;
- if (ObjectStore != null && objectStoreTailAddress.HasValue)
- defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value;
defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow;
}
@@ -189,13 +169,10 @@ public override async Task TakeOnDemandCheckpointAsync(DateTimeOffset entryTime,
// Necessary to take a checkpoint because the latest checkpoint is before entryTime
var result = await TakeCheckpointAsync(defaultDatabase, logger: Logger);
- var storeTailAddress = result.Item1;
- var objectStoreTailAddress = result.Item2;
+ var storeTailAddress = result;
if (storeTailAddress.HasValue)
defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value;
- if (ObjectStore != null && objectStoreTailAddress.HasValue)
- defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value;
defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow;
}
@@ -222,13 +199,10 @@ public override async Task TaskCheckpointBasedOnAofSizeLimitAsync(long aofSizeLi
{
var result = await TakeCheckpointAsync(defaultDatabase, logger: logger, token: token);
- var storeTailAddress = result.Item1;
- var objectStoreTailAddress = result.Item2;
+ var storeTailAddress = result;
if (storeTailAddress.HasValue)
defaultDatabase.LastSaveStoreTailAddress = storeTailAddress.Value;
- if (ObjectStore != null && objectStoreTailAddress.HasValue)
- defaultDatabase.LastSaveObjectStoreTailAddress = objectStoreTailAddress.Value;
defaultDatabase.LastSaveTime = DateTimeOffset.UtcNow;
}
@@ -307,8 +281,8 @@ public override void ExpiredKeyDeletionScan() =>
ExpiredKeyDeletionScan(defaultDatabase);
///
- public override void StartObjectSizeTrackers(CancellationToken token = default) =>
- ObjectStoreSizeTracker?.Start(token);
+ public override void StartSizeTrackers(CancellationToken token = default) =>
+ SizeTracker?.Start(token);
///
public override void Reset(int dbId = 0)
@@ -320,10 +294,7 @@ public override void Reset(int dbId = 0)
///
public override void ResetRevivificationStats()
- {
- MainStore.ResetRevivificationStats();
- ObjectStore?.ResetRevivificationStats();
- }
+ => Store.ResetRevivificationStats();
///
public override void EnqueueCommit(AofEntryType entryType, long version, int dbId = 0)
@@ -379,7 +350,7 @@ public override FunctionsState CreateFunctionsState(int dbId = 0, byte respProto
{
ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0);
- return new(AppendOnlyFile, VersionMap, StoreWrapper, null, ObjectStoreSizeTracker, Logger, respProtocolVersion);
+ return new(AppendOnlyFile, VersionMap, StoreWrapper, null, SizeTracker, Logger, respProtocolVersion);
}
private async Task TryPauseCheckpointsContinuousAsync(int dbId,
@@ -401,9 +372,7 @@ private async Task TryPauseCheckpointsContinuousAsync(int dbId,
public override (long numExpiredKeysFound, long totalRecordsScanned) ExpiredKeyDeletionScan(int dbId)
{
ArgumentOutOfRangeException.ThrowIfNotEqual(dbId, 0);
- var (k1, t1) = MainStoreExpiredKeyDeletionScan(DefaultDatabase);
- var (k2, t2) = StoreWrapper.serverOptions.DisableObjects ? (0, 0) : ObjectStoreExpiredKeyDeletionScan(DefaultDatabase);
- return (k1 + k2, t1 + t2);
+ return StoreExpiredKeyDeletionScan(DefaultDatabase);
}
public override (HybridLogScanMetrics mainStore, HybridLogScanMetrics objectStore)[] CollectHybridLogStats() => [CollectHybridLogStatsForDb(defaultDatabase)];
diff --git a/libs/server/GarnetDatabase.cs b/libs/server/GarnetDatabase.cs
index f09b1016681..31b5eccd604 100644
--- a/libs/server/GarnetDatabase.cs
+++ b/libs/server/GarnetDatabase.cs
@@ -8,11 +8,8 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
-
- using ObjectStoreAllocator = ObjectAllocator>;
- using ObjectStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Represents a logical database in Garnet
@@ -31,14 +28,9 @@ public class GarnetDatabase : IDisposable
public int Id { get; }
///
- /// Main Store
- ///
- public TsavoriteKV MainStore { get; }
-
- ///
- /// Object Store
+ /// Store
///
- public TsavoriteKV ObjectStore { get; }
+ public TsavoriteKV Store { get; }
///
/// Epoch instance used by server
@@ -51,9 +43,9 @@ public class GarnetDatabase : IDisposable
public StateMachineDriver StateMachineDriver { get; }
///
- /// Size Tracker for Object Store
+ /// Size Tracker
///
- public CacheSizeTracker ObjectStoreSizeTracker { get; }
+ public CacheSizeTracker SizeTracker { get; }
///
/// Device used for AOF logging
@@ -71,29 +63,19 @@ public class GarnetDatabase : IDisposable
public WatchVersionMap VersionMap { get; }
///
- /// Tail address of main store log at last save
+ /// Tail address of store log at last save
///
public long LastSaveStoreTailAddress;
- ///
- /// Tail address of object store log at last save
- ///
- public long LastSaveObjectStoreTailAddress;
-
///
/// Last time checkpoint of database was taken
///
public DateTimeOffset LastSaveTime;
///
- /// True if database's main store index has maxed-out
+ /// True if database's store index has maxed-out
///
- public bool MainStoreIndexMaxedOut;
-
- ///
- /// True if database's object store index has maxed-out
- ///
- public bool ObjectStoreIndexMaxedOut;
+ public bool StoreIndexMaxedOut;
///
/// Reader-Writer lock for database checkpointing
@@ -103,59 +85,47 @@ public class GarnetDatabase : IDisposable
///
/// Storage session intended for store-wide object collection operations
///
- internal StorageSession ObjectStoreCollectionDbStorageSession;
+ internal StorageSession StoreCollectionDbStorageSession;
///
- /// Storage session intended for main-store expired key deletion operations
+ /// Storage session intended for store expired key deletion operations
///
- internal StorageSession MainStoreExpiredKeyDeletionDbStorageSession;
-
- ///
- /// Storage session intended for object-store expired key deletion operations
- ///
- internal StorageSession ObjectStoreExpiredKeyDeletionDbStorageSession;
-
+ internal StorageSession StoreExpiredKeyDeletionDbStorageSession;
internal StorageSession HybridLogStatScanStorageSession;
bool disposed = false;
- public GarnetDatabase(int id, TsavoriteKV mainStore,
- TsavoriteKV objectStore,
+ public GarnetDatabase(int id, TsavoriteKV store,
LightEpoch epoch, StateMachineDriver stateMachineDriver,
- CacheSizeTracker objectStoreSizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile,
- bool mainStoreIndexMaxedOut, bool objectStoreIndexMaxedOut) : this()
+ CacheSizeTracker sizeTracker, IDevice aofDevice, TsavoriteLog appendOnlyFile,
+ bool storeIndexMaxedOut) : this()
{
Id = id;
- MainStore = mainStore;
- ObjectStore = objectStore;
+ Store = store;
Epoch = epoch;
StateMachineDriver = stateMachineDriver;
- ObjectStoreSizeTracker = objectStoreSizeTracker;
+ SizeTracker = sizeTracker;
AofDevice = aofDevice;
AppendOnlyFile = appendOnlyFile;
- MainStoreIndexMaxedOut = mainStoreIndexMaxedOut;
- ObjectStoreIndexMaxedOut = objectStoreIndexMaxedOut;
+ StoreIndexMaxedOut = storeIndexMaxedOut;
}
public GarnetDatabase(int id, GarnetDatabase srcDb, bool enableAof, bool copyLastSaveData = false) : this()
{
Id = id;
- MainStore = srcDb.MainStore;
- ObjectStore = srcDb.ObjectStore;
+ Store = srcDb.Store;
Epoch = srcDb.Epoch;
StateMachineDriver = srcDb.StateMachineDriver;
- ObjectStoreSizeTracker = srcDb.ObjectStoreSizeTracker;
+ SizeTracker = srcDb.SizeTracker;
AofDevice = enableAof ? srcDb.AofDevice : null;
AppendOnlyFile = enableAof ? srcDb.AppendOnlyFile : null;
- MainStoreIndexMaxedOut = srcDb.MainStoreIndexMaxedOut;
- ObjectStoreIndexMaxedOut = srcDb.ObjectStoreIndexMaxedOut;
+ StoreIndexMaxedOut = srcDb.StoreIndexMaxedOut;
if (copyLastSaveData)
{
LastSaveTime = srcDb.LastSaveTime;
LastSaveStoreTailAddress = srcDb.LastSaveStoreTailAddress;
- LastSaveObjectStoreTailAddress = srcDb.LastSaveObjectStoreTailAddress;
}
}
@@ -163,7 +133,6 @@ public GarnetDatabase()
{
VersionMap = new WatchVersionMap(DefaultVersionMapSize);
LastSaveStoreTailAddress = 0;
- LastSaveObjectStoreTailAddress = 0;
LastSaveTime = DateTimeOffset.FromUnixTimeSeconds(0);
}
@@ -177,20 +146,18 @@ public void Dispose()
// Wait for checkpoints to complete and disable checkpointing
CheckpointingLock.CloseLock();
- MainStore?.Dispose();
- ObjectStore?.Dispose();
+ Store?.Dispose();
AofDevice?.Dispose();
AppendOnlyFile?.Dispose();
- ObjectStoreCollectionDbStorageSession?.Dispose();
- MainStoreExpiredKeyDeletionDbStorageSession?.Dispose();
- ObjectStoreExpiredKeyDeletionDbStorageSession?.Dispose();
+ StoreCollectionDbStorageSession?.Dispose();
+ StoreExpiredKeyDeletionDbStorageSession?.Dispose();
- if (ObjectStoreSizeTracker != null)
+ if (SizeTracker != null)
{
// If tracker has previously started, wait for it to stop
- if (!ObjectStoreSizeTracker.TryPreventStart())
+ if (!SizeTracker.TryPreventStart())
{
- while (!ObjectStoreSizeTracker.Stopped)
+ while (!SizeTracker.Stopped)
Thread.Yield();
}
}
diff --git a/libs/server/GarnetUnifiedStoreOutput.cs b/libs/server/GarnetUnifiedStoreOutput.cs
new file mode 100644
index 00000000000..18e12464659
--- /dev/null
+++ b/libs/server/GarnetUnifiedStoreOutput.cs
@@ -0,0 +1,48 @@
+// Copyright (c) Microsoft Corporation.
+// Licensed under the MIT license.
+
+using Tsavorite.core;
+
+namespace Garnet.server
+{
+ ///
+ /// Output type used by Garnet unified store.
+ /// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller).
+ /// That is in order to justify transferring data in this struct through the Tsavorite storage layer.
+ ///
+ public struct GarnetUnifiedStoreOutput
+ {
+ ///
+ /// Span byte and memory
+ ///
+ public SpanByteAndMemory SpanByteAndMemory;
+
+ ///
+ /// Output header
+ ///
+ public OutputHeader Header;
+
+ ///
+ /// Output flags
+ ///
+ public OutputFlags OutputFlags;
+
+ ///
+ /// True if output flag RemoveKey is set
+ ///
+ public readonly bool HasRemoveKey =>
+ (OutputFlags & OutputFlags.RemoveKey) == OutputFlags.RemoveKey;
+
+ public GarnetUnifiedStoreOutput() => SpanByteAndMemory = new(null);
+
+ public GarnetUnifiedStoreOutput(SpanByteAndMemory span) => SpanByteAndMemory = span;
+
+ public static unsafe GarnetUnifiedStoreOutput FromPinnedPointer(byte* pointer, int length)
+ => new(new SpanByteAndMemory() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length) });
+
+ public void ConvertToHeap()
+ {
+ // Does not convert to heap when going pending, because we immediately complete pending operations for unified store.
+ }
+ }
+}
\ No newline at end of file
diff --git a/libs/server/InputHeader.cs b/libs/server/InputHeader.cs
index 81596288cb2..824623ed3a1 100644
--- a/libs/server/InputHeader.cs
+++ b/libs/server/InputHeader.cs
@@ -427,6 +427,127 @@ public unsafe int DeserializeFrom(byte* src)
}
}
+ ///
+ /// Header for Garnet Main Store inputs
+ ///
+ public struct UnifiedStoreInput : IStoreInput
+ {
+ ///
+ /// Common input header for Garnet
+ ///
+ public RespInputHeader header;
+
+ ///
+ /// Argument for generic usage by command implementation
+ ///
+ public long arg1;
+
+ ///
+ /// Session parse state
+ ///
+ public SessionParseState parseState;
+
+ ///
+ /// Create a new instance of UnifiedStoreInput
+ ///
+ /// Command
+ /// Flags
+ /// General-purpose argument
+ public UnifiedStoreInput(RespCommand cmd, RespInputFlags flags = 0, long arg1 = 0)
+ {
+ this.header = new RespInputHeader(cmd, flags);
+ this.arg1 = arg1;
+ }
+
+ ///
+ /// Create a new instance of UnifiedStoreInput
+ ///
+ /// Command
+ /// Flags
+ /// General-purpose argument
+ public UnifiedStoreInput(ushort cmd, byte flags = 0, long arg1 = 0) :
+ this((RespCommand)cmd, (RespInputFlags)flags, arg1)
+
+ {
+ }
+
+ ///
+ /// Create a new instance of UnifiedStoreInput
+ ///
+ /// Command
+ /// Parse state
+ /// General-purpose argument
+ /// Flags
+ public UnifiedStoreInput(RespCommand cmd, ref SessionParseState parseState, long arg1 = 0, RespInputFlags flags = 0) : this(cmd, flags, arg1)
+ {
+ this.parseState = parseState;
+ }
+
+ ///
+ /// Create a new instance of UnifiedStoreInput
+ ///
+ /// Command
+ /// Parse state
+ /// First command argument index in parse state
+ /// General-purpose argument
+ /// Flags
+ public UnifiedStoreInput(RespCommand cmd, ref SessionParseState parseState, int startIdx, long arg1 = 0, RespInputFlags flags = 0) : this(cmd, flags, arg1)
+ {
+ this.parseState = parseState.Slice(startIdx);
+ }
+
+ ///
+ public int SerializedLength => header.SpanByte.TotalSize
+ + sizeof(long) // arg1
+ + parseState.GetSerializedLength();
+
+ ///
+ public unsafe int CopyTo(byte* dest, int length)
+ {
+ Debug.Assert(length >= this.SerializedLength);
+
+ var curr = dest;
+
+ // Serialize header
+ header.SpanByte.SerializeTo(curr);
+ curr += header.SpanByte.TotalSize;
+
+ // Serialize arg1
+ *(long*)curr = arg1;
+ curr += sizeof(long);
+
+ // Serialize parse state
+ var remainingLength = length - (int)(curr - dest);
+ var len = parseState.SerializeTo(curr, remainingLength);
+ curr += len;
+
+ // Serialize length
+ return (int)(curr - dest);
+ }
+
+ ///
+ public unsafe int DeserializeFrom(byte* src)
+ {
+ var curr = src;
+
+ // Deserialize header
+ var header = PinnedSpanByte.FromLengthPrefixedPinnedPointer(curr);
+ ref var h = ref Unsafe.AsRef(header.ToPointer());
+ curr += header.TotalSize;
+ this.header = h;
+
+ // Deserialize arg1
+ arg1 = *(long*)curr;
+ curr += sizeof(long);
+
+ // Deserialize parse state
+ var len = parseState.DeserializeFrom(curr);
+ curr += len;
+
+ return (int)(curr - src);
+ }
+ }
+
///
/// Header for Garnet CustomProcedure inputs
///
@@ -494,22 +615,4 @@ public unsafe int DeserializeFrom(byte* src)
return len;
}
}
-
- ///
- /// Object output header (sometimes used as footer)
- ///
- [StructLayout(LayoutKind.Explicit, Size = Size)]
- public struct ObjectOutputHeader
- {
- ///
- /// Expected size of this object
- ///
- public const int Size = 4;
-
- ///
- /// Some result of operation (e.g., number of items added successfully)
- ///
- [FieldOffset(0)]
- public int result1;
- }
}
\ No newline at end of file
diff --git a/libs/server/Lua/LuaRunner.Functions.cs b/libs/server/Lua/LuaRunner.Functions.cs
index 03d9717113c..56246344555 100644
--- a/libs/server/Lua/LuaRunner.Functions.cs
+++ b/libs/server/Lua/LuaRunner.Functions.cs
@@ -227,9 +227,7 @@ internal int UnsafeRunPreambleForSession(nint luaStatePtr)
if (txnMode)
{
- txnKeyEntries.AddKey(key, false, Tsavorite.core.LockType.Exclusive);
- if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull)
- txnKeyEntries.AddKey(key, true, Tsavorite.core.LockType.Exclusive);
+ txnKeyEntries.AddKey(key, LockType.Exclusive);
}
// Equivalent to KEYS[i+1] = key
diff --git a/libs/server/Lua/LuaRunner.cs b/libs/server/Lua/LuaRunner.cs
index 310f2ba6fe0..0a6c6ccbd31 100644
--- a/libs/server/Lua/LuaRunner.cs
+++ b/libs/server/Lua/LuaRunner.cs
@@ -263,7 +263,9 @@ public unsafe LuaRunner(
delegate* unmanaged[Cdecl] garnetCall;
if (txnMode)
{
- txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.transactionalContext, respServerSession.storageSession.objectStoreTransactionalContext);
+ txnKeyEntries = new TxnKeyEntries(16, respServerSession.storageSession.transactionalContext,
+ respServerSession.storageSession.objectStoreTransactionalContext,
+ respServerSession.storageSession.unifiedStoreTransactionalContext);
garnetCall = &LuaRunnerTrampolines.GarnetCallWithTransaction;
}
@@ -1237,9 +1239,7 @@ public unsafe object RunForRunner(string[] keys = null, string[] argv = null)
foreach (var key in keys)
{
var _key = scratchBufferBuilder.CreateArgSlice(key);
- txnKeyEntries.AddKey(_key, false, Tsavorite.core.LockType.Exclusive);
- if (!respServerSession.storageSession.objectStoreTransactionalContext.IsNull)
- txnKeyEntries.AddKey(_key, true, Tsavorite.core.LockType.Exclusive);
+ txnKeyEntries.AddKey(_key, Tsavorite.core.LockType.Exclusive);
}
adapter = new(scratchBufferBuilder);
diff --git a/libs/server/Metrics/Info/GarnetInfoMetrics.cs b/libs/server/Metrics/Info/GarnetInfoMetrics.cs
index 59ea3a4150f..fec79cdeb57 100644
--- a/libs/server/Metrics/Info/GarnetInfoMetrics.cs
+++ b/libs/server/Metrics/Info/GarnetInfoMetrics.cs
@@ -18,9 +18,7 @@ class GarnetInfoMetrics
.Where(e => e switch
{
InfoMetricsType.STOREHASHTABLE => false,
- InfoMetricsType.OBJECTSTOREHASHTABLE => false,
InfoMetricsType.STOREREVIV => false,
- InfoMetricsType.OBJECTSTOREREVIV => false,
InfoMetricsType.HLOGSCAN => false,
_ => true
})];
@@ -31,11 +29,8 @@ class GarnetInfoMetrics
MetricsItem[] replicationInfo = null;
MetricsItem[] statsInfo = null;
MetricsItem[][] storeInfo = null;
- MetricsItem[][] objectStoreInfo = null;
MetricsItem[][] storeHashDistrInfo = null;
- MetricsItem[][] objectStoreHashDistrInfo = null;
MetricsItem[][] storeRevivInfo = null;
- MetricsItem[][] objectStoreRevivInfo = null;
MetricsItem[][] persistenceInfo = null;
MetricsItem[] clientsInfo = null;
MetricsItem[] keyspaceInfo = null;
@@ -65,21 +60,13 @@ private void PopulateServerInfo(StoreWrapper storeWrapper)
private void PopulateMemoryInfo(StoreWrapper storeWrapper)
{
- var main_store_index_size = 0L;
- var main_store_log_memory_size = 0L;
- var main_store_read_cache_size = 0L;
- long total_main_store_size;
-
- var disableObj = storeWrapper.serverOptions.DisableObjects;
-
- var initialSize = disableObj ? -1L : 0L;
- var object_store_index_size = initialSize;
- var object_store_log_memory_size = initialSize;
- var object_store_read_cache_log_memory_size = initialSize;
- var object_store_heap_memory_target_size = initialSize;
- var object_store_heap_memory_size = initialSize;
- var object_store_read_cache_heap_memory_size = initialSize;
- var total_object_store_size = initialSize;
+ var store_index_size = 0L;
+ var store_log_memory_size = 0L;
+ var store_read_cache_size = 0L;
+ var store_heap_memory_target_size = 0L;
+ var store_heap_memory_size = 0L;
+ var store_read_cache_heap_memory_size = 0L;
+ long total_store_size;
var enableAof = storeWrapper.serverOptions.EnableAOF;
var aof_log_memory_size = enableAof ? 0 : -1L;
@@ -88,31 +75,19 @@ private void PopulateMemoryInfo(StoreWrapper storeWrapper)
foreach (var db in databases)
{
- main_store_index_size += db.MainStore.IndexSize * 64;
- main_store_log_memory_size += db.MainStore.Log.MemorySizeBytes;
- main_store_read_cache_size += db.MainStore.ReadCache?.MemorySizeBytes ?? 0;
+ store_index_size += db.Store.IndexSize * 64;
+ store_log_memory_size += db.Store.Log.MemorySizeBytes;
+ store_read_cache_size += db.Store.ReadCache?.MemorySizeBytes ?? 0;
aof_log_memory_size += db.AppendOnlyFile?.MemorySizeBytes ?? 0;
- if (!disableObj)
- {
- object_store_index_size += db.ObjectStore.IndexSize * 64;
- object_store_log_memory_size += db.ObjectStore.Log.MemorySizeBytes;
- object_store_read_cache_log_memory_size += db.ObjectStore.ReadCache?.MemorySizeBytes ?? 0;
- object_store_heap_memory_target_size += db.ObjectStoreSizeTracker?.mainLogTracker.TargetSize ?? 0;
- object_store_heap_memory_size += db.ObjectStoreSizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0;
- object_store_read_cache_heap_memory_size += db.ObjectStoreSizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0;
- }
+ store_heap_memory_target_size += db.SizeTracker?.mainLogTracker.TargetSize ?? 0;
+ store_heap_memory_size += db.SizeTracker?.mainLogTracker.LogHeapSizeBytes ?? 0;
+ store_read_cache_heap_memory_size += db.SizeTracker?.readCacheTracker?.LogHeapSizeBytes ?? 0;
}
- total_main_store_size = main_store_index_size + main_store_log_memory_size + main_store_read_cache_size;
-
- if (!disableObj)
- {
- total_object_store_size = object_store_index_size + object_store_log_memory_size +
- object_store_read_cache_log_memory_size + object_store_heap_memory_size +
- object_store_read_cache_heap_memory_size;
- }
+ total_store_size = store_index_size + store_log_memory_size + store_read_cache_size +
+ store_heap_memory_size + store_read_cache_heap_memory_size;
var gcMemoryInfo = GC.GetGCMemoryInfo();
var gcAvailableMemory = gcMemoryInfo.TotalCommittedBytes - gcMemoryInfo.HeapSizeBytes;
@@ -144,17 +119,13 @@ private void PopulateMemoryInfo(StoreWrapper storeWrapper)
new("gc_heap_bytes", gcMemoryInfo.HeapSizeBytes.ToString()),
new("gc_managed_memory_bytes_excluding_heap", gcAvailableMemory.ToString()),
new("gc_fragmented_bytes", gcMemoryInfo.FragmentedBytes.ToString()),
- new("main_store_index_size", main_store_index_size.ToString()),
- new("main_store_log_memory_size", main_store_log_memory_size.ToString()),
- new("main_store_read_cache_size", main_store_read_cache_size.ToString()),
- new("total_main_store_size", total_main_store_size.ToString()),
- new("object_store_index_size", object_store_index_size.ToString()),
- new("object_store_log_memory_size", object_store_log_memory_size.ToString()),
- new("object_store_heap_memory_target_size", object_store_heap_memory_target_size.ToString()),
- new("object_store_heap_memory_size", object_store_heap_memory_size.ToString()),
- new("object_store_read_cache_log_memory_size", object_store_read_cache_log_memory_size.ToString()),
- new("object_store_read_cache_heap_memory_size", object_store_read_cache_heap_memory_size.ToString()),
- new("total_object_store_size", total_object_store_size.ToString()),
+ new("store_index_size", store_index_size.ToString()),
+ new("store_log_memory_size", store_log_memory_size.ToString()),
+ new("store_read_cache_size", store_read_cache_size.ToString()),
+ new("total_main_store_size", total_store_size.ToString()),
+ new("store_heap_memory_target_size", store_heap_memory_target_size.ToString()),
+ new("store_heap_memory_size", store_heap_memory_size.ToString()),
+ new("store_read_cache_heap_memory_size", store_read_cache_heap_memory_size.ToString()),
new("aof_memory_size", aof_log_memory_size.ToString())
];
}
@@ -179,8 +150,6 @@ private void PopulateReplicationInfo(StoreWrapper storeWrapper)
new("second_repl_offset", "N/A"),
new("store_current_safe_aof_address", "N/A"),
new("store_recovered_safe_aof_address", "N/A"),
- new("object_store_current_safe_aof_address", "N/A"),
- new("object_store_recovered_safe_aof_address", "N/A")
];
}
else
@@ -244,60 +213,25 @@ private void PopulateStoreStats(StoreWrapper storeWrapper)
private MetricsItem[] GetDatabaseStoreStats(StoreWrapper storeWrapper, GarnetDatabase db) =>
[
- new($"CurrentVersion", db.MainStore.CurrentVersion.ToString()),
- new($"LastCheckpointedVersion", db.MainStore.LastCheckpointedVersion.ToString()),
- new($"SystemState", db.MainStore.SystemState.ToString()),
- new($"IndexSize", db.MainStore.IndexSize.ToString()),
- new($"LogDir", storeWrapper.serverOptions.LogDir),
- new($"Log.BeginAddress", db.MainStore.Log.BeginAddress.ToString()),
- new($"Log.BufferSize", db.MainStore.Log.BufferSize.ToString()),
- new($"Log.EmptyPageCount", db.MainStore.Log.EmptyPageCount.ToString()),
- new($"Log.MinEmptyPageCount", db.MainStore.Log.MinEmptyPageCount.ToString()),
- new($"Log.HeadAddress", db.MainStore.Log.HeadAddress.ToString()),
- new($"Log.MemorySizeBytes", db.MainStore.Log.MemorySizeBytes.ToString()),
- new($"Log.SafeReadOnlyAddress", db.MainStore.Log.SafeReadOnlyAddress.ToString()),
- new($"Log.TailAddress", db.MainStore.Log.TailAddress.ToString()),
- new($"ReadCache.BeginAddress", db.MainStore.ReadCache?.BeginAddress.ToString() ?? "N/A"),
- new($"ReadCache.BufferSize", db.MainStore.ReadCache?.BufferSize.ToString() ?? "N/A"),
- new($"ReadCache.EmptyPageCount", db.MainStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"),
- new($"ReadCache.HeadAddress", db.MainStore.ReadCache?.HeadAddress.ToString() ?? "N/A"),
- new($"ReadCache.MemorySizeBytes", db.MainStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"),
- new($"ReadCache.TailAddress", db.MainStore.ReadCache?.TailAddress.ToString() ?? "N/A"),
- ];
-
- private void PopulateObjectStoreStats(StoreWrapper storeWrapper)
- {
- var databases = storeWrapper.GetDatabasesSnapshot();
-
- objectStoreInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][];
- foreach (var db in databases)
- {
- var storeStats = GetDatabaseObjectStoreStats(storeWrapper, db);
- objectStoreInfo[db.Id] = storeStats;
- }
- }
-
- private MetricsItem[] GetDatabaseObjectStoreStats(StoreWrapper storeWrapper, GarnetDatabase db) =>
- [
- new($"CurrentVersion", db.ObjectStore.CurrentVersion.ToString()),
- new($"LastCheckpointedVersion", db.ObjectStore.LastCheckpointedVersion.ToString()),
- new($"SystemState", db.ObjectStore.SystemState.ToString()),
- new($"IndexSize", db.ObjectStore.IndexSize.ToString()),
+ new($"CurrentVersion", db.Store.CurrentVersion.ToString()),
+ new($"LastCheckpointedVersion", db.Store.LastCheckpointedVersion.ToString()),
+ new($"SystemState", db.Store.SystemState.ToString()),
+ new($"IndexSize", db.Store.IndexSize.ToString()),
new($"LogDir", storeWrapper.serverOptions.LogDir),
- new($"Log.BeginAddress", db.ObjectStore.Log.BeginAddress.ToString()),
- new($"Log.BufferSize", db.ObjectStore.Log.BufferSize.ToString()),
- new($"Log.EmptyPageCount", db.ObjectStore.Log.EmptyPageCount.ToString()),
- new($"Log.MinEmptyPageCount", db.ObjectStore.Log.MinEmptyPageCount.ToString()),
- new($"Log.HeadAddress", db.ObjectStore.Log.HeadAddress.ToString()),
- new($"Log.MemorySizeBytes", db.ObjectStore.Log.MemorySizeBytes.ToString()),
- new($"Log.SafeReadOnlyAddress", db.ObjectStore.Log.SafeReadOnlyAddress.ToString()),
- new($"Log.TailAddress", db.ObjectStore.Log.TailAddress.ToString()),
- new($"ReadCache.BeginAddress", db.ObjectStore.ReadCache?.BeginAddress.ToString() ?? "N/A"),
- new($"ReadCache.BufferSize", db.ObjectStore.ReadCache?.BufferSize.ToString() ?? "N/A"),
- new($"ReadCache.EmptyPageCount", db.ObjectStore.ReadCache?.EmptyPageCount.ToString() ?? "N/A"),
- new($"ReadCache.HeadAddress", db.ObjectStore.ReadCache?.HeadAddress.ToString() ?? "N/A"),
- new($"ReadCache.MemorySizeBytes", db.ObjectStore.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"),
- new($"ReadCache.TailAddress", db.ObjectStore.ReadCache?.TailAddress.ToString() ?? "N/A"),
+ new($"Log.BeginAddress", db.Store.Log.BeginAddress.ToString()),
+ new($"Log.BufferSize", db.Store.Log.BufferSize.ToString()),
+ new($"Log.EmptyPageCount", db.Store.Log.EmptyPageCount.ToString()),
+ new($"Log.MinEmptyPageCount", db.Store.Log.MinEmptyPageCount.ToString()),
+ new($"Log.HeadAddress", db.Store.Log.HeadAddress.ToString()),
+ new($"Log.MemorySizeBytes", db.Store.Log.MemorySizeBytes.ToString()),
+ new($"Log.SafeReadOnlyAddress", db.Store.Log.SafeReadOnlyAddress.ToString()),
+ new($"Log.TailAddress", db.Store.Log.TailAddress.ToString()),
+ new($"ReadCache.BeginAddress", db.Store.ReadCache?.BeginAddress.ToString() ?? "N/A"),
+ new($"ReadCache.BufferSize", db.Store.ReadCache?.BufferSize.ToString() ?? "N/A"),
+ new($"ReadCache.EmptyPageCount", db.Store.ReadCache?.EmptyPageCount.ToString() ?? "N/A"),
+ new($"ReadCache.HeadAddress", db.Store.ReadCache?.HeadAddress.ToString() ?? "N/A"),
+ new($"ReadCache.MemorySizeBytes", db.Store.ReadCache?.MemorySizeBytes.ToString() ?? "N/A"),
+ new($"ReadCache.TailAddress", db.Store.ReadCache?.TailAddress.ToString() ?? "N/A"),
];
private void PopulateStoreHashDistribution(StoreWrapper storeWrapper)
@@ -307,18 +241,7 @@ private void PopulateStoreHashDistribution(StoreWrapper storeWrapper)
storeHashDistrInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][];
foreach (var db in databases)
{
- storeHashDistrInfo[db.Id] = [new("", db.MainStore.DumpDistribution())];
- }
- }
-
- private void PopulateObjectStoreHashDistribution(StoreWrapper storeWrapper)
- {
- var databases = storeWrapper.GetDatabasesSnapshot();
-
- objectStoreHashDistrInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][];
- foreach (var db in databases)
- {
- objectStoreHashDistrInfo[db.Id] = [new("", db.ObjectStore.DumpDistribution())];
+ storeHashDistrInfo[db.Id] = [new("", db.Store.DumpDistribution())];
}
}
@@ -329,18 +252,7 @@ private void PopulateStoreRevivInfo(StoreWrapper storeWrapper)
storeRevivInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][];
foreach (var db in databases)
{
- storeRevivInfo[db.Id] = [new("", db.MainStore.DumpRevivificationStats())];
- }
- }
-
- private void PopulateObjectStoreRevivInfo(StoreWrapper storeWrapper)
- {
- var databases = storeWrapper.GetDatabasesSnapshot();
-
- objectStoreRevivInfo = new MetricsItem[storeWrapper.MaxDatabaseId + 1][];
- foreach (var db in databases)
- {
- objectStoreRevivInfo[db.Id] = [new("", db.ObjectStore.DumpRevivificationStats())];
+ storeRevivInfo[db.Id] = [new("", db.Store.DumpRevivificationStats())];
}
}
@@ -430,11 +342,8 @@ public static string GetSectionHeader(InfoMetricsType infoType, int dbId)
InfoMetricsType.REPLICATION => "Replication",
InfoMetricsType.STATS => "Stats",
InfoMetricsType.STORE => $"MainStore_DB_{dbId}",
- InfoMetricsType.OBJECTSTORE => $"ObjectStore_DB_{dbId}",
InfoMetricsType.STOREHASHTABLE => $"MainStoreHashTableDistribution_DB_{dbId}",
- InfoMetricsType.OBJECTSTOREHASHTABLE => $"ObjectStoreHashTableDistribution_DB_{dbId}",
InfoMetricsType.STOREREVIV => $"MainStoreDeletedRecordRevivification_DB_{dbId}",
- InfoMetricsType.OBJECTSTOREREVIV => $"ObjectStoreDeletedRecordRevivification_DB_{dbId}",
InfoMetricsType.PERSISTENCE => $"Persistence_DB_{dbId}",
InfoMetricsType.CLIENTS => "Clients",
InfoMetricsType.KEYSPACE => "Keyspace",
@@ -495,32 +404,14 @@ private void GetRespInfo(InfoMetricsType section, int dbId, StoreWrapper storeWr
PopulateStoreStats(storeWrapper);
GetSectionRespInfo(header, storeInfo[dbId], sbResponse);
return;
- case InfoMetricsType.OBJECTSTORE:
- if (storeWrapper.serverOptions.DisableObjects)
- return;
- PopulateObjectStoreStats(storeWrapper);
- GetSectionRespInfo(header, objectStoreInfo[dbId], sbResponse);
- return;
case InfoMetricsType.STOREHASHTABLE:
PopulateStoreHashDistribution(storeWrapper);
GetSectionRespInfo(header, storeHashDistrInfo[dbId], sbResponse);
return;
- case InfoMetricsType.OBJECTSTOREHASHTABLE:
- if (storeWrapper.serverOptions.DisableObjects)
- return;
- PopulateObjectStoreHashDistribution(storeWrapper);
- GetSectionRespInfo(header, objectStoreHashDistrInfo[dbId], sbResponse);
- return;
case InfoMetricsType.STOREREVIV:
PopulateStoreRevivInfo(storeWrapper);
GetSectionRespInfo(header, storeRevivInfo[dbId], sbResponse);
return;
- case InfoMetricsType.OBJECTSTOREREVIV:
- if (storeWrapper.serverOptions.DisableObjects)
- return;
- PopulateObjectStoreRevivInfo(storeWrapper);
- GetSectionRespInfo(header, objectStoreRevivInfo[dbId], sbResponse);
- return;
case InfoMetricsType.PERSISTENCE:
if (!storeWrapper.serverOptions.EnableAOF)
return;
@@ -591,27 +482,12 @@ private MetricsItem[] GetMetricInternal(InfoMetricsType section, int dbId, Store
case InfoMetricsType.STORE:
PopulateStoreStats(storeWrapper);
return storeInfo[dbId];
- case InfoMetricsType.OBJECTSTORE:
- if (storeWrapper.serverOptions.DisableObjects)
- return null;
- PopulateObjectStoreStats(storeWrapper);
- return objectStoreInfo[dbId];
case InfoMetricsType.STOREHASHTABLE:
PopulateStoreHashDistribution(storeWrapper);
return storeHashDistrInfo[dbId];
- case InfoMetricsType.OBJECTSTOREHASHTABLE:
- if (storeWrapper.serverOptions.DisableObjects)
- return null;
- PopulateObjectStoreHashDistribution(storeWrapper);
- return objectStoreHashDistrInfo[dbId];
case InfoMetricsType.STOREREVIV:
PopulateStoreRevivInfo(storeWrapper);
return storeRevivInfo[dbId];
- case InfoMetricsType.OBJECTSTOREREVIV:
- if (storeWrapper.serverOptions.DisableObjects)
- return null;
- PopulateObjectStoreRevivInfo(storeWrapper);
- return objectStoreRevivInfo[dbId];
case InfoMetricsType.PERSISTENCE:
if (!storeWrapper.serverOptions.EnableAOF)
return null;
diff --git a/libs/server/Metrics/Info/InfoHelp.cs b/libs/server/Metrics/Info/InfoHelp.cs
index 47cc24a3460..656f71c5015 100644
--- a/libs/server/Metrics/Info/InfoHelp.cs
+++ b/libs/server/Metrics/Info/InfoHelp.cs
@@ -23,11 +23,8 @@ public static List GetInfoTypeHelpMessage()
$"{nameof(InfoMetricsType.REPLICATION)}: Replication info.",
$"{nameof(InfoMetricsType.STATS)}: General server operational stats.",
$"{nameof(InfoMetricsType.STORE)}: Main store operational information.",
- $"{nameof(InfoMetricsType.OBJECTSTORE)}: Object store operational information.",
$"{nameof(InfoMetricsType.STOREHASHTABLE)}: Hash table distribution info for main store (expensive, not returned by default).",
- $"{nameof(InfoMetricsType.OBJECTSTOREHASHTABLE)}: Hash table distribution info for object store (expensive, not returned by default).",
$"{nameof(InfoMetricsType.STOREREVIV)}: Revivification info for deleted records in main store (not returned by default).",
- $"{nameof(InfoMetricsType.OBJECTSTOREREVIV)}: Record revivification info for deleted records in object store (not returned by default).",
$"{nameof(InfoMetricsType.PERSISTENCE)}: Persistence related information (i.e. Checkpoint and AOF).",
$"{nameof(InfoMetricsType.CLIENTS)}: Information related to client connections.",
$"{nameof(InfoMetricsType.KEYSPACE)}: Database related statistics.",
diff --git a/libs/server/Objects/Hash/HashObject.cs b/libs/server/Objects/Hash/HashObject.cs
index 1fdd4885095..0eaf71735b0 100644
--- a/libs/server/Objects/Hash/HashObject.cs
+++ b/libs/server/Objects/Hash/HashObject.cs
@@ -189,7 +189,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
if (input.header.type != GarnetObjectType.Hash)
{
//Indicates when there is an incorrect type
- output.OutputFlags |= ObjectStoreOutputFlags.WrongType;
+ output.OutputFlags |= OutputFlags.WrongType;
output.SpanByteAndMemory.Length = 0;
return true;
}
@@ -264,7 +264,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
memorySizeChange = this.HeapMemorySize - previousMemorySize;
if (hash.Count == 0)
- output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey;
+ output.OutputFlags |= OutputFlags.RemoveKey;
return true;
}
diff --git a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs
index b7fbae5a2f5..78a8f0011e7 100644
--- a/libs/server/Objects/ItemBroker/CollectionItemBroker.cs
+++ b/libs/server/Objects/ItemBroker/CollectionItemBroker.cs
@@ -529,16 +529,17 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp
{
Debug.Assert(storageSession.txnManager.state == TxnState.None);
createTransaction = true;
- storageSession.txnManager.SaveKeyEntryToLock(asKey, true, LockType.Exclusive);
+ storageSession.txnManager.AddTransactionStoreTypes(TransactionStoreTypes.Object | TransactionStoreTypes.Unified);
+ storageSession.txnManager.SaveKeyEntryToLock(asKey, LockType.Exclusive);
if (command == RespCommand.BLMOVE)
- storageSession.txnManager.SaveKeyEntryToLock(dstKey, true, LockType.Exclusive);
+ storageSession.txnManager.SaveKeyEntryToLock(dstKey, LockType.Exclusive);
_ = storageSession.txnManager.Run(true);
}
- var transactionalContext = storageSession.txnManager.TransactionalContext;
var objectTransactionalContext = storageSession.txnManager.ObjectStoreTransactionalContext;
+ var unifiedTransactionalContext = storageSession.txnManager.UnifiedStoreTransactionalContext;
try
{
@@ -641,8 +642,8 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp
if (isSuccessful && listObj.LnkList.Count == 0)
{
- _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None,
- ref transactionalContext, ref objectTransactionalContext);
+ _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, ExpireOption.None,
+ ref unifiedTransactionalContext);
}
return isSuccessful;
@@ -655,8 +656,8 @@ private unsafe bool TryGetResult(byte[] key, StorageSession storageSession, Resp
if (isSuccessful && sortedSetObj.Count() == 0)
{
- _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, StoreType.Object, ExpireOption.None,
- ref transactionalContext, ref objectTransactionalContext);
+ _ = storageSession.EXPIRE(asKey, TimeSpan.Zero, out _, ExpireOption.None,
+ ref unifiedTransactionalContext);
}
return isSuccessful;
diff --git a/libs/server/Objects/List/ListObject.cs b/libs/server/Objects/List/ListObject.cs
index b3ed0878043..1a9b333d51c 100644
--- a/libs/server/Objects/List/ListObject.cs
+++ b/libs/server/Objects/List/ListObject.cs
@@ -136,7 +136,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
if (input.header.type != GarnetObjectType.List)
{
// Indicates an incorrect type of key
- output.OutputFlags |= ObjectStoreOutputFlags.WrongType;
+ output.OutputFlags |= OutputFlags.WrongType;
output.SpanByteAndMemory.Length = 0;
return true;
}
@@ -190,7 +190,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
memorySizeChange = this.HeapMemorySize - previousMemorySize;
if (list.Count == 0)
- output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey;
+ output.OutputFlags |= OutputFlags.RemoveKey;
return true;
}
diff --git a/libs/server/Objects/Set/SetObject.cs b/libs/server/Objects/Set/SetObject.cs
index 19c09061fe9..76a27c0959f 100644
--- a/libs/server/Objects/Set/SetObject.cs
+++ b/libs/server/Objects/Set/SetObject.cs
@@ -130,7 +130,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
if (input.header.type != GarnetObjectType.Set)
{
// Indicates an incorrect type of key
- output.OutputFlags |= ObjectStoreOutputFlags.WrongType;
+ output.OutputFlags |= OutputFlags.WrongType;
output.SpanByteAndMemory.Length = 0;
return true;
}
@@ -172,7 +172,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
memorySizeChange = this.HeapMemorySize - prevMemorySize;
if (Set.Count == 0)
- output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey;
+ output.OutputFlags |= OutputFlags.RemoveKey;
return true;
}
diff --git a/libs/server/Objects/SortedSet/SortedSetObject.cs b/libs/server/Objects/SortedSet/SortedSetObject.cs
index 21c5b4c16ce..8adda584c6b 100644
--- a/libs/server/Objects/SortedSet/SortedSetObject.cs
+++ b/libs/server/Objects/SortedSet/SortedSetObject.cs
@@ -327,7 +327,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
if (header.type != GarnetObjectType.SortedSet)
{
// Indicates an incorrect type of key
- output.OutputFlags |= ObjectStoreOutputFlags.WrongType;
+ output.OutputFlags |= OutputFlags.WrongType;
output.SpanByteAndMemory.Length = 0;
return true;
}
@@ -421,7 +421,7 @@ public override bool Operate(ref ObjectInput input, ref GarnetObjectStoreOutput
memorySizeChange = this.HeapMemorySize - prevMemorySize;
if (sortedSetDict.Count == 0)
- output.OutputFlags |= ObjectStoreOutputFlags.RemoveKey;
+ output.OutputFlags |= OutputFlags.RemoveKey;
return true;
}
diff --git a/libs/server/Objects/Types/GarnetObject.cs b/libs/server/Objects/Types/GarnetObject.cs
index b9fc4abf680..c99f91aa957 100644
--- a/libs/server/Objects/Types/GarnetObject.cs
+++ b/libs/server/Objects/Types/GarnetObject.cs
@@ -74,8 +74,6 @@ internal static bool NeedToCreate(RespInputHeader header)
HashOperation.HCOLLECT => false,
_ => true,
},
- GarnetObjectType.Expire => false,
- GarnetObjectType.Persist => false,
GarnetObjectType.Migrate => false,
_ => true,
};
diff --git a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs
index 0547c6042b4..856fa5225b5 100644
--- a/libs/server/Objects/Types/GarnetObjectStoreOutput.cs
+++ b/libs/server/Objects/Types/GarnetObjectStoreOutput.cs
@@ -1,33 +1,10 @@
// Copyright (c) Microsoft Corporation.
// Licensed under the MIT license.
-using System;
using Tsavorite.core;
namespace Garnet.server
{
- ///
- /// Flags for object store outputs.
- ///
- [Flags]
- public enum ObjectStoreOutputFlags : byte
- {
- ///
- /// No flags set
- ///
- None = 0,
-
- ///
- /// Remove key
- ///
- RemoveKey = 1,
-
- ///
- /// Wrong type of object
- ///
- WrongType = 1 << 1,
- }
-
///
/// Output type used by Garnet object store.
/// Any field / property added to this struct must be set in the back-end (IFunctions) and used in the front-end (GarnetApi caller).
@@ -46,28 +23,30 @@ public struct GarnetObjectStoreOutput
public IGarnetObject GarnetObject;
///
- /// Object header
+ /// Output header
///
- public ObjectOutputHeader Header;
+ public OutputHeader Header;
///
/// Output flags
///
- public ObjectStoreOutputFlags OutputFlags;
+ public OutputFlags OutputFlags;
///
/// True if output flag WrongType is set
///
- public readonly bool HasWrongType => (OutputFlags & ObjectStoreOutputFlags.WrongType) == ObjectStoreOutputFlags.WrongType;
+ public readonly bool HasWrongType =>
+ (OutputFlags & OutputFlags.WrongType) == OutputFlags.WrongType;
///
/// True if output flag RemoveKey is set
///
- public readonly bool HasRemoveKey => (OutputFlags & ObjectStoreOutputFlags.RemoveKey) == ObjectStoreOutputFlags.RemoveKey;
+ public readonly bool HasRemoveKey =>
+ (OutputFlags & OutputFlags.RemoveKey) == OutputFlags.RemoveKey;
public GarnetObjectStoreOutput() => SpanByteAndMemory = new(null);
- public GarnetObjectStoreOutput(SpanByteAndMemory spam) => SpanByteAndMemory = spam;
+ public GarnetObjectStoreOutput(SpanByteAndMemory span) => SpanByteAndMemory = span;
public static unsafe GarnetObjectStoreOutput FromPinnedPointer(byte* pointer, int length)
=> new(new SpanByteAndMemory() { SpanByte = PinnedSpanByte.FromPinnedPointer(pointer, length) });
diff --git a/libs/server/Objects/Types/GarnetObjectType.cs b/libs/server/Objects/Types/GarnetObjectType.cs
index 2960e75a5d3..df5d77bcb4a 100644
--- a/libs/server/Objects/Types/GarnetObjectType.cs
+++ b/libs/server/Objects/Types/GarnetObjectType.cs
@@ -43,21 +43,6 @@ public enum GarnetObjectType : byte
///
DelIfExpIm = 0xf7,
- ///
- /// Special type indicating PEXPIRE command
- ///
- PExpire = 0xf8,
-
- ///
- /// Special type indicating EXPIRETIME command
- ///
- ExpireTime = 0xf9,
-
- ///
- /// Special type indicating PEXPIRETIME command
- ///
- PExpireTime = 0xfa,
-
///
/// Indicating a Custom Object command
///
@@ -68,20 +53,10 @@ public enum GarnetObjectType : byte
///
PTtl = 0xfc,
- ///
- /// Special type indicating PERSIST command
- ///
- Persist = 0xfd,
-
///
/// Special type indicating TTL command
///
Ttl = 0xfe,
-
- ///
- /// Special type indicating EXPIRE command
- ///
- Expire = 0xff,
}
public static class GarnetObjectTypeExtensions
diff --git a/libs/server/OutputHeader.cs b/libs/server/OutputHeader.cs
new file mode 100644
index 00000000000..bb45baf3ba8
--- /dev/null
+++ b/libs/server/OutputHeader.cs
@@ -0,0 +1,48 @@
+// Copyright (c) Microsoft Corporation.
+// Licensed under the MIT license.
+
+using System;
+using System.Runtime.InteropServices;
+
+namespace Garnet.server
+{
+ ///
+ /// Flags for store outputs.
+ ///
+ [Flags]
+ public enum OutputFlags : byte
+ {
+ ///
+ /// No flags set
+ ///
+ None = 0,
+
+ ///
+ /// Remove key
+ ///
+ RemoveKey = 1,
+
+ ///
+ /// Wrong type of value
+ ///
+ WrongType = 1 << 1,
+ }
+
+ ///
+ /// Object output header (sometimes used as footer)
+ ///
+ [StructLayout(LayoutKind.Explicit, Size = Size)]
+ public struct OutputHeader
+ {
+ ///
+ /// Expected size of this struct
+ ///
+ public const int Size = 4;
+
+ ///
+ /// Some result of operation (e.g., number of items added successfully)
+ ///
+ [FieldOffset(0)]
+ public int result1;
+ }
+}
\ No newline at end of file
diff --git a/libs/server/Providers/GarnetProvider.cs b/libs/server/Providers/GarnetProvider.cs
index e8b53202805..b72d4160f01 100644
--- a/libs/server/Providers/GarnetProvider.cs
+++ b/libs/server/Providers/GarnetProvider.cs
@@ -8,13 +8,13 @@
namespace Garnet.server
{
- using MainStoreAllocator = SpanByteAllocator>;
- using MainStoreFunctions = StoreFunctions;
+ using StoreAllocator = ObjectAllocator>;
+ using StoreFunctions = StoreFunctions;
///
/// Session provider for Garnet
///
- public sealed class GarnetProvider : TsavoriteKVProviderBase
+ public sealed class GarnetProvider : TsavoriteKVProviderBase
{
readonly StoreWrapper storeWrapper;
diff --git a/libs/server/Resp/ArrayCommands.cs b/libs/server/Resp/ArrayCommands.cs
index 8d6dda4dc75..4dab5fca5a0 100644
--- a/libs/server/Resp/ArrayCommands.cs
+++ b/libs/server/Resp/ArrayCommands.cs
@@ -194,10 +194,11 @@ private bool NetworkDEL(ref TGarnetApi storageApi)
where TGarnetApi : IGarnetApi
{
int keysDeleted = 0;
+
for (int c = 0; c < parseState.Count; c++)
{
var key = parseState.GetArgSliceByRef(c);
- var status = storageApi.DELETE(key, StoreType.All);
+ var status = storageApi.DELETE(key);
// This is only an approximate count because the deletion of a key on disk is performed as a blind tombstone append
if (status == GarnetStatus.OK)
@@ -438,16 +439,22 @@ private bool NetworkTYPE(ref TGarnetApi storageApi)
// TYPE key
var keySlice = parseState.GetArgSliceByRef(0);
- var status = storageApi.GetKeyType(keySlice, out var typeName);
+ // Prepare input
+ var input = new UnifiedStoreInput(RespCommand.TYPE);
+
+ // Prepare GarnetUnifiedStoreOutput output
+ var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr));
+
+
+ var status = storageApi.TYPE(keySlice, ref input, ref output);
if (status == GarnetStatus.OK)
{
- while (!RespWriteUtils.TryWriteSimpleString(typeName, ref dcurr, dend))
- SendAndReset();
+ ProcessOutput(output.SpanByteAndMemory);
}
else
{
- while (!RespWriteUtils.TryWriteSimpleString("none"u8, ref dcurr, dend))
+ while (!RespWriteUtils.TryWriteSimpleString(CmdStrings.none, ref dcurr, dend))
SendAndReset();
}
diff --git a/libs/server/Resp/BasicCommands.cs b/libs/server/Resp/BasicCommands.cs
index 5accfc9f569..922b58b7214 100644
--- a/libs/server/Resp/BasicCommands.cs
+++ b/libs/server/Resp/BasicCommands.cs
@@ -1160,23 +1160,21 @@ private bool NetworkCOMMAND_GETKEYS()
}
var cmdName = parseState.GetString(0);
- bool cmdFound = RespCommandsInfo.TryGetRespCommandInfo(cmdName, out var cmdInfo, true, true, logger) ||
- storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out cmdInfo);
- if (!cmdFound)
- {
+ // Try to parse command and get its simplified info
+ if (!TryGetSimpleCommandInfo(cmdName, out var simpleCmdInfo))
return AbortWithErrorMessage(CmdStrings.RESP_INVALID_COMMAND_SPECIFIED);
- }
- if (cmdInfo.KeySpecifications == null || cmdInfo.KeySpecifications.Length == 0)
- {
+ // If command has no key specifications, abort with error
+ if (simpleCmdInfo.KeySpecs == null || simpleCmdInfo.KeySpecs.Length == 0)
return AbortWithErrorMessage(CmdStrings.RESP_COMMAND_HAS_NO_KEY_ARGS);
- }
- parseState.TryExtractKeysFromSpecs(cmdInfo.KeySpecifications, out var keys);
+ // Extract command keys from parse state and key specification
+ // An offset is applied to the parse state, as the command (and possibly subcommand) are included in the parse state.
+ var slicedParseState = parseState.Slice(simpleCmdInfo.IsSubCommand ? 2 : 1);
+ var keys = slicedParseState.ExtractCommandKeys(simpleCmdInfo);
-
- while (!RespWriteUtils.TryWriteArrayLength(keys.Count, ref dcurr, dend))
+ while (!RespWriteUtils.TryWriteArrayLength(keys.Length, ref dcurr, dend))
SendAndReset();
foreach (var key in keys)
@@ -1199,35 +1197,35 @@ private bool NetworkCOMMAND_GETKEYSANDFLAGS()
}
var cmdName = parseState.GetString(0);
- bool cmdFound = RespCommandsInfo.TryGetRespCommandInfo(cmdName, out var cmdInfo, true, true, logger) ||
- storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out cmdInfo);
- if (!cmdFound)
- {
+ // Try to parse command and get its simplified info
+ if (!TryGetSimpleCommandInfo(cmdName, out var simpleCmdInfo))
return AbortWithErrorMessage(CmdStrings.RESP_INVALID_COMMAND_SPECIFIED);
- }
- if (cmdInfo.KeySpecifications == null || cmdInfo.KeySpecifications.Length == 0)
- {
+ // If command has no key specifications, abort with error
+ if (simpleCmdInfo.KeySpecs == null || simpleCmdInfo.KeySpecs.Length == 0)
return AbortWithErrorMessage(CmdStrings.RESP_COMMAND_HAS_NO_KEY_ARGS);
- }
- parseState.TryExtractKeysAndFlagsFromSpecs(cmdInfo.KeySpecifications, out var keys, out var flags);
+ // Extract command keys from parse state and key specification
+ // An offset is applied to the parse state, as the command (and possibly subcommand) are included in the parse state.
+ var slicedParseState = parseState.Slice(simpleCmdInfo.IsSubCommand ? 2 : 1);
+ var keysAndFlags = slicedParseState.ExtractCommandKeysAndFlags(simpleCmdInfo);
- while (!RespWriteUtils.TryWriteArrayLength(keys.Count, ref dcurr, dend))
+ while (!RespWriteUtils.TryWriteArrayLength(keysAndFlags.Length, ref dcurr, dend))
SendAndReset();
- for (int i = 0; i < keys.Count; i++)
+ for (var i = 0; i < keysAndFlags.Length; i++)
{
while (!RespWriteUtils.TryWriteArrayLength(2, ref dcurr, dend))
SendAndReset();
- while (!RespWriteUtils.TryWriteBulkString(keys[i].Span, ref dcurr, dend))
+ while (!RespWriteUtils.TryWriteBulkString(keysAndFlags[i].Item1.Span, ref dcurr, dend))
SendAndReset();
- WriteSetLength(flags[i].Length);
+ var flags = EnumUtils.GetEnumDescriptions(keysAndFlags[i].Item2);
+ WriteSetLength(flags.Length);
- foreach (var flag in flags[i])
+ foreach (var flag in flags)
{
while (!RespWriteUtils.TryWriteBulkString(Encoding.ASCII.GetBytes(flag), ref dcurr, dend))
SendAndReset();
@@ -1433,12 +1431,17 @@ private bool NetworkMemoryUsage(ref TGarnetApi storageApi)
}
}
- var status = storageApi.MemoryUsageForKey(key, out var memoryUsage);
+ // Prepare input
+ var input = new UnifiedStoreInput(RespCommand.MEMORY_USAGE);
+
+ // Prepare GarnetUnifiedStoreOutput output
+ var output = GarnetUnifiedStoreOutput.FromPinnedPointer(dcurr, (int)(dend - dcurr));
+
+ var status = storageApi.MEMORYUSAGE(key, ref input, ref output);
if (status == GarnetStatus.OK)
{
- while (!RespWriteUtils.TryWriteInt32((int)memoryUsage, ref dcurr, dend))
- SendAndReset();
+ ProcessOutput(output.SpanByteAndMemory);
}
else
{
@@ -1747,6 +1750,29 @@ bool ParseGETAndKey(ref PinnedSpanByte key)
return true;
}
+ private bool TryGetSimpleCommandInfo(string cmdName, out SimpleRespCommandInfo simpleCmdInfo)
+ {
+ simpleCmdInfo = SimpleRespCommandInfo.Default;
+
+ // Try to parse known command from name and obtain its command info
+ if (!Enum.TryParse(cmdName, true, out var cmd) ||
+ !RespCommandsInfo.TryGetSimpleRespCommandInfo(cmd, out simpleCmdInfo, logger))
+ {
+ // If we no known command or info was found, attempt to find custom command
+ if (storeWrapper.customCommandManager.TryGetCustomCommandInfo(cmdName, out var cmdInfo))
+ {
+ cmdInfo.PopulateSimpleCommandInfo(ref simpleCmdInfo);
+ }
+ else
+ {
+ // No matching command was found
+ return false;
+ }
+ }
+
+ return true;
+ }
+
static void SetResult(int c, ref int firstPending, ref (GarnetStatus, SpanByteAndMemory)[] outputArr,
GarnetStatus status, SpanByteAndMemory output)
{
diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs
index e0a4a29eb77..426f5677c65 100644
--- a/libs/server/Resp/CmdStrings.cs
+++ b/libs/server/Resp/CmdStrings.cs
@@ -357,6 +357,7 @@ static partial class CmdStrings
public static ReadOnlySpan hash => "hash"u8;
public static ReadOnlySpan STRING => "STRING"u8;
public static ReadOnlySpan stringt => "string"u8;
+ public static ReadOnlySpan none => "none"u8;
///
/// Register object types
diff --git a/libs/server/Resp/GarnetDatabaseSession.cs b/libs/server/Resp/GarnetDatabaseSession.cs
index eebe240656b..bb56abebf7b 100644
--- a/libs/server/Resp/GarnetDatabaseSession.cs
+++ b/libs/server/Resp/GarnetDatabaseSession.cs
@@ -4,16 +4,22 @@
namespace Garnet.server
{
using BasicGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
BasicContext,
+ ObjectAllocator>>,
+ BasicContext,
ObjectAllocator>>>;
using TransactionalGarnetApi = GarnetApi,
- SpanByteAllocator>>,
+ /* MainStoreFunctions */ StoreFunctions,
+ ObjectAllocator>>,
TransactionalContext,
+ ObjectAllocator>>,
+ TransactionalContext,
ObjectAllocator>>>;
///