From e92d8c1a3178d22820d6573def979508b0d7f3e3 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:28:47 -0800 Subject: [PATCH 01/32] expose diskless replication parameters --- libs/host/Configuration/Options.cs | 11 ++++++++++- libs/host/defaults.conf | 6 ++++++ libs/server/Servers/GarnetServerOptions.cs | 10 ++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs index 402ca9dd1c..45045255a5 100644 --- a/libs/host/Configuration/Options.cs +++ b/libs/host/Configuration/Options.cs @@ -9,7 +9,6 @@ using System.Net; using System.Net.Sockets; using System.Reflection; -using System.Runtime.InteropServices; using System.Security.Cryptography.X509Certificates; using CommandLine; using Garnet.server; @@ -407,6 +406,14 @@ internal sealed class Options [Option("on-demand-checkpoint", Required = false, HelpText = "Used with main-memory replication model. Take on demand checkpoint to avoid missing data when attaching")] public bool? OnDemandCheckpoint { get; set; } + [OptionValidation] + [Option("repl-diskless-sync", Required = false, HelpText = "Whether diskless replication is enabled or not.")] + public bool? ReplicaDisklessSync { get; set; } + + [IntRangeValidation(0, int.MaxValue)] + [Option("repl-diskless-sync-delay", Required = false, Default = 5, HelpText = "Delay in diskless replication sync in seconds. =0: Immediately start diskless replication sync.")] + public int ReplicaDisklessSyncDelay { get; set; } + [OptionValidation] [Option("aof-null-device", Required = false, HelpText = "With main-memory replication, use null device for AOF. Ensures no disk IO, but can cause data loss during replication.")] public bool? UseAofNullDevice { get; set; } @@ -764,6 +771,8 @@ public GarnetServerOptions GetServerOptions(ILogger logger = null) ReplicationOffsetMaxLag = ReplicationOffsetMaxLag, MainMemoryReplication = MainMemoryReplication.GetValueOrDefault(), OnDemandCheckpoint = OnDemandCheckpoint.GetValueOrDefault(), + ReplicaDisklessSync = ReplicaDisklessSync.GetValueOrDefault(), + ReplicaDisklessSyncDelay = ReplicaDisklessSyncDelay, UseAofNullDevice = UseAofNullDevice.GetValueOrDefault(), ClusterUsername = ClusterUsername, ClusterPassword = ClusterPassword, diff --git a/libs/host/defaults.conf b/libs/host/defaults.conf index e0c7543795..a8a32143ff 100644 --- a/libs/host/defaults.conf +++ b/libs/host/defaults.conf @@ -302,6 +302,12 @@ /* Used with main-memory replication model. Take on demand checkpoint to avoid missing data when attaching */ "OnDemandCheckpoint" : false, + /* Whether diskless replication is enabled or not */ + "ReplicaDisklessSync" : false, + + /* Delay in diskless replication sync in seconds. =0: Immediately start diskless replication sync. */ + "ReplicaDisklessSyncDelay" : 5, + /* With main-memory replication, use null device for AOF. Ensures no disk IO, but can cause data loss during replication. */ "UseAofNullDevice" : false, diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index e90b778a36..c922461252 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -329,6 +329,16 @@ public class GarnetServerOptions : ServerOptions /// public bool OnDemandCheckpoint = false; + /// + /// Whether diskless replication is enabled or not. + /// + public bool ReplicaDisklessSync = false; + + /// + /// Delay in diskless replication sync in seconds. =0: Immediately start diskless replication sync. + /// + public int ReplicaDisklessSyncDelay = 5; + /// /// With main-memory replication, whether we use null device for AOF. Ensures no disk IO, but can cause data loss during replication. /// From 1bd4927dee8e16c09b496fd99bce349c31972b75 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:39:55 -0800 Subject: [PATCH 02/32] refactor/cleanup legacy ReplicaSyncSession --- .../PrimaryOps/ReplicaSyncSession.cs | 70 +++++++++++-------- .../ReplicaOps/ReplicaReceiveCheckpoint.cs | 53 +++++++------- 2 files changed, 66 insertions(+), 57 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index de853ca91c..0e2ca027d8 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -15,20 +15,30 @@ namespace Garnet.cluster { - internal sealed class ReplicaSyncSession(StoreWrapper storeWrapper, ClusterProvider clusterProvider, string remoteNodeId, string remote_primary_replid, CheckpointEntry remoteEntry, long replicaAofBeginAddress, long replicaAofTailAddress, ILogger logger = null) : IDisposable + internal sealed partial class ReplicaSyncSession( + StoreWrapper storeWrapper, + ClusterProvider clusterProvider, + SyncMetadata replicaSyncMetadata = null, + string replicaNodeId = null, + string replicaAssignedPrimaryId = null, + CheckpointEntry replicaCheckpointEntry = null, + long replicaAofBeginAddress = 0, + long replicaAofTailAddress = 0, + ILogger logger = null) : IDisposable { readonly StoreWrapper storeWrapper = storeWrapper; readonly ClusterProvider clusterProvider = clusterProvider; - readonly CancellationTokenSource ctsCheckpointRetrievalSession = new(); - private SectorAlignedBufferPool bufferPool = null; - private readonly SemaphoreSlim semaphore = new(0); + public readonly SyncMetadata replicaSyncMetadata = replicaSyncMetadata; + readonly CancellationTokenSource cts = new(); + SectorAlignedBufferPool bufferPool = null; + readonly SemaphoreSlim semaphore = new(0); - public readonly string remoteNodeId = remoteNodeId; - public readonly string remote_primary_replid = remote_primary_replid; + public readonly string replicaNodeId = replicaNodeId; + public readonly string replicaAssignedPrimaryId = replicaAssignedPrimaryId; private readonly long replicaAofBeginAddress = replicaAofBeginAddress; private readonly long replicaAofTailAddress = replicaAofTailAddress; - private readonly CheckpointEntry remoteEntry = remoteEntry; + private readonly CheckpointEntry replicaCheckpointEntry = replicaCheckpointEntry; private readonly ILogger logger = logger; @@ -36,8 +46,8 @@ internal sealed class ReplicaSyncSession(StoreWrapper storeWrapper, ClusterProvi public void Dispose() { - ctsCheckpointRetrievalSession.Cancel(); - ctsCheckpointRetrievalSession.Dispose(); + cts.Cancel(); + cts.Dispose(); semaphore?.Dispose(); bufferPool?.Free(); } @@ -52,11 +62,11 @@ public async Task SendCheckpoint() var storeCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Main); var objectStoreCkptManager = clusterProvider.GetReplicationLogCheckpointManager(StoreType.Object); var current = clusterProvider.clusterManager.CurrentConfig; - var (address, port) = current.GetWorkerAddressFromNodeId(remoteNodeId); + var (address, port) = current.GetWorkerAddressFromNodeId(replicaNodeId); if (address == null || port == -1) { - errorMsg = $"PRIMARY-ERR don't know about replicaId: {remoteNodeId}"; + errorMsg = $"PRIMARY-ERR don't know about replicaId: {replicaNodeId}"; logger?.LogError("{errorMsg}", errorMsg); return false; } @@ -75,7 +85,7 @@ public async Task SendCheckpoint() try { logger?.LogInformation("Replica replicaId:{replicaId} requesting checkpoint replicaStoreVersion:{replicaStoreVersion} replicaObjectStoreVersion:{replicaObjectStoreVersion}", - remoteNodeId, remoteEntry.metadata.storeVersion, remoteEntry.metadata.objectStoreVersion); + replicaNodeId, replicaCheckpointEntry.metadata.storeVersion, replicaCheckpointEntry.metadata.objectStoreVersion); gcs.Connect((int)clusterProvider.clusterManager.GetClusterTimeout().TotalMilliseconds); retry: @@ -86,13 +96,13 @@ public async Task SendCheckpoint() // 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(remoteEntry.metadata.storePrimaryReplId) && remoteEntry.metadata.storePrimaryReplId.Equals(localEntry.metadata.storePrimaryReplId); - var sameObjectStoreCheckpointHistory = !string.IsNullOrEmpty(remoteEntry.metadata.objectStorePrimaryReplId) && remoteEntry.metadata.objectStorePrimaryReplId.Equals(localEntry.metadata.objectStorePrimaryReplId); + 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 - var skipLocalMainStoreCheckpoint = localEntry.metadata.storeHlogToken == default || (sameMainStoreCheckpointHistory && localEntry.metadata.storeVersion == remoteEntry.metadata.storeVersion); - var skipLocalObjectStoreCheckpoint = clusterProvider.serverOptions.DisableObjects || localEntry.metadata.objectStoreHlogToken == default || (sameObjectStoreCheckpointHistory && localEntry.metadata.objectStoreVersion == remoteEntry.metadata.objectStoreVersion); + var skipLocalMainStoreCheckpoint = localEntry.metadata.storeHlogToken == default || (sameMainStoreCheckpointHistory && localEntry.metadata.storeVersion == replicaCheckpointEntry.metadata.storeVersion); + var skipLocalObjectStoreCheckpoint = clusterProvider.serverOptions.DisableObjects || localEntry.metadata.objectStoreHlogToken == default || (sameObjectStoreCheckpointHistory && localEntry.metadata.objectStoreVersion == replicaCheckpointEntry.metadata.objectStoreVersion); LogFileInfo hlog_size = default; long index_size = -1; @@ -195,26 +205,26 @@ public async Task SendCheckpoint() var recoverFromRemote = !skipLocalMainStoreCheckpoint || !skipLocalObjectStoreCheckpoint; var replayAOF = false; - var RecoveredReplicationOffset = localEntry.GetMinAofCoveredAddress(); - var beginAddress = RecoveredReplicationOffset; + var checkpointAofBeginAddress = localEntry.GetMinAofCoveredAddress(); + var beginAddress = checkpointAofBeginAddress; if (!recoverFromRemote) { // If replica is ahead of this primary it will force itself to forget and start syncing from RecoveredReplicationOffset - if (replicaAofBeginAddress > ReplicationManager.kFirstValidAofAddress && replicaAofBeginAddress > RecoveredReplicationOffset) + if (replicaAofBeginAddress > ReplicationManager.kFirstValidAofAddress && replicaAofBeginAddress > checkpointAofBeginAddress) { logger?.LogInformation( "ReplicaSyncSession: replicaAofBeginAddress {replicaAofBeginAddress} > PrimaryCheckpointRecoveredReplicationOffset {RecoveredReplicationOffset}, cannot use remote AOF", - replicaAofBeginAddress, RecoveredReplicationOffset); + replicaAofBeginAddress, checkpointAofBeginAddress); } else { // Tail address cannot be behind the recovered address since above we checked replicaAofBeginAddress and it appears after RecoveredReplicationOffset // unless we are performing MainMemoryReplication // TODO: shouldn't we use the remote cEntry's tail address here since replica will recover to that? - if (replicaAofTailAddress < RecoveredReplicationOffset && !clusterProvider.serverOptions.MainMemoryReplication) + if (replicaAofTailAddress < checkpointAofBeginAddress && !clusterProvider.serverOptions.MainMemoryReplication) { - logger?.LogCritical("ReplicaSyncSession replicaAofTail {replicaAofTailAddress} < canServeFromAofAddress {RecoveredReplicationOffset}", replicaAofTailAddress, RecoveredReplicationOffset); - throw new Exception($"ReplicaSyncSession replicaAofTail {replicaAofTailAddress} < canServeFromAofAddress {RecoveredReplicationOffset}"); + logger?.LogCritical("ReplicaSyncSession replicaAofTail {replicaAofTailAddress} < canServeFromAofAddress {RecoveredReplicationOffset}", replicaAofTailAddress, checkpointAofBeginAddress); + throw new Exception($"ReplicaSyncSession replicaAofTail {replicaAofTailAddress} < canServeFromAofAddress {checkpointAofBeginAddress}"); } // If we are behind this primary we need to decide until where to replay @@ -226,16 +236,16 @@ public async Task SendCheckpoint() } // Replay only if records not included in checkpoint - if (replayUntilAddress > RecoveredReplicationOffset) + if (replayUntilAddress > checkpointAofBeginAddress) { logger?.LogInformation("ReplicaSyncSession: have to replay remote AOF from {beginAddress} until {untilAddress}", beginAddress, replayUntilAddress); replayAOF = true; // Bound replayUntilAddress to ReplicationOffset2 to avoid replaying divergent history only if connecting replica was attached to old primary if (!string.IsNullOrEmpty(clusterProvider.replicationManager.PrimaryReplId2) && - clusterProvider.replicationManager.PrimaryReplId2.Equals(remote_primary_replid) && + clusterProvider.replicationManager.PrimaryReplId2.Equals(replicaAssignedPrimaryId) && replayUntilAddress > clusterProvider.replicationManager.ReplicationOffset2) replayUntilAddress = clusterProvider.replicationManager.ReplicationOffset2; - RecoveredReplicationOffset = replayUntilAddress; + checkpointAofBeginAddress = replayUntilAddress; } } } @@ -249,7 +259,7 @@ public async Task SendCheckpoint() clusterProvider.replicationManager.PrimaryReplId, localEntry.ToByteArray(), beginAddress, - RecoveredReplicationOffset).ConfigureAwait(false); + checkpointAofBeginAddress).ConfigureAwait(false); var syncFromAofAddress = long.Parse(resp); // Assert that AOF address the replica will be requesting can be served, except in case of: @@ -280,9 +290,9 @@ public async Task SendCheckpoint() // We have already added the iterator for the covered address above but replica might request an address // that is ahead of the covered address so we should start streaming from that address in order not to // introduce duplicate insertions. - if (!clusterProvider.replicationManager.TryAddReplicationTask(remoteNodeId, syncFromAofAddress, out aofSyncTaskInfo)) + if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaNodeId, syncFromAofAddress, out aofSyncTaskInfo)) throw new GarnetException("Failed trying to try update replication task"); - if (!clusterProvider.replicationManager.TryConnectToReplica(remoteNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) + if (!clusterProvider.replicationManager.TryConnectToReplica(replicaNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) throw new GarnetException("Failed connecting to replica for aofSync"); } catch (Exception ex) @@ -347,7 +357,7 @@ public void AcquireCheckpointEntry(out CheckpointEntry cEntry, out AofSyncTaskIn // Enqueue AOF sync task with startAofAddress to prevent future AOF truncations // and check if truncation has happened in between retrieving the latest checkpoint and enqueuing the aofSyncTask - if (clusterProvider.replicationManager.TryAddReplicationTask(remoteNodeId, startAofAddress, out aofSyncTaskInfo)) + if (clusterProvider.replicationManager.TryAddReplicationTask(replicaNodeId, startAofAddress, out aofSyncTaskInfo)) break; // Unlock last checkpoint because associated startAofAddress is no longer available diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs index e872b9961b..2612fb8e28 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaReceiveCheckpoint.cs @@ -40,7 +40,6 @@ public bool TryBeginReplicate(ClusterSession session, string nodeid, bool backgr // Ensure two replicate commands do not execute at the same time. if (!replicateLock.TryWriteLock()) { - errorMessage = "ERR Replicate already in progress"u8; errorMessage = CmdStrings.RESP_ERR_GENERIC_CANNOT_ACQUIRE_REPLICATE_LOCK; return false; } @@ -228,32 +227,6 @@ CheckpointFileType.OBJ_STORE_SNAPSHOT or } } - private IDevice GetStoreHLogDevice() - { - 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 logFactory.Get(new FileDescriptor("Store", "hlog")); - } - return null; - } - - private 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; - } - /// /// Check if device needs to be initialized with a specifi segment size depending on the checkpoint file type /// @@ -292,6 +265,32 @@ public IDevice GetInitializedSegmentFileDevice(Guid token, CheckpointFileType ty if (ShouldInitialize(type)) device.Initialize(segmentSize: 1L << clusterProvider.serverOptions.SegmentSizeBits()); return device; + + IDevice GetStoreHLogDevice() + { + 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 logFactory.Get(new FileDescriptor("Store", "hlog")); + } + 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; + } } /// From 1131c57da565319998892b4207df592d4055af1a Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:44:06 -0800 Subject: [PATCH 03/32] add interface to support diskless replication session and aof tasks --- .../Server/Replication/CheckpointEntry.cs | 1 + .../Replication/PrimaryOps/AofTaskStore.cs | 109 ++++++++++++++++++ .../PrimaryOps/ReplicaSyncSessionTaskStore.cs | 71 ++++++++++-- .../PrimaryOps/ReplicationPrimaryAofSync.cs | 3 + 4 files changed, 177 insertions(+), 7 deletions(-) diff --git a/libs/cluster/Server/Replication/CheckpointEntry.cs b/libs/cluster/Server/Replication/CheckpointEntry.cs index b8a7a776ab..1a707d6383 100644 --- a/libs/cluster/Server/Replication/CheckpointEntry.cs +++ b/libs/cluster/Server/Replication/CheckpointEntry.cs @@ -118,6 +118,7 @@ public byte[] ToByteArray() public static CheckpointEntry FromByteArray(byte[] serialized) { + if (serialized.Length == 0) return null; var ms = new MemoryStream(serialized); var reader = new BinaryReader(ms); var cEntry = new CheckpointEntry diff --git a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs index abfbfa6aa3..fffbf21048 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs @@ -217,6 +217,115 @@ public bool TryAddReplicationTask(string remoteNodeId, long startAddress, out Ao return success; } + public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, long startAddress) + { + var current = clusterProvider.clusterManager.CurrentConfig; + var success = true; + if (startAddress == 0) startAddress = ReplicationManager.kFirstValidAofAddress; + + // First iterate through all sync sessions and add an AOF sync task + // All tasks will be + foreach (var rss in replicaSyncSessions) + { + if (rss == null) continue; + var replicaNodeId = rss.replicaSyncMetadata.originNodeId; + var (address, port) = current.GetWorkerAddressFromNodeId(replicaNodeId); + + try + { + rss.AddAofSyncTask(new AofSyncTaskInfo( + clusterProvider, + this, + current.LocalNodeId, + replicaNodeId, + new GarnetClientSession( + address, + port, + clusterProvider.replicationManager.GetAofSyncNetworkBufferSettings, + clusterProvider.replicationManager.GetNetworkPool, + tlsOptions: clusterProvider.serverOptions.TlsOptions?.TlsClientOptions, + authUsername: clusterProvider.ClusterUsername, + authPassword: clusterProvider.ClusterPassword, + logger: logger), + startAddress, + logger)); + } + catch (Exception ex) + { + logger?.LogWarning(ex, "{method} creating AOF sync task for {replicaNodeId} failed", nameof(TryAddReplicationTasks), replicaNodeId); + return false; + } + } + + _lock.WriteLock(); + try + { + if (_disposed) return false; + + // Fail adding the task if truncation has happened + if (startAddress < TruncatedUntil) + { + logger?.LogWarning("{method} failed to add tasks for AOF sync {startAddress} {truncatedUntil}", nameof(TryAddReplicationTasks), startAddress, TruncatedUntil); + return false; + } + + foreach (var rss in replicaSyncSessions) + { + if (rss == null) continue; + + var added = false; + // Find if AOF sync task already exists + for (var i = 0; i < numTasks; i++) + { + var t = tasks[i]; + Debug.Assert(t != null); + if (t.remoteNodeId == rss.replicaNodeId) + { + tasks[i] = rss.GetAofSyncTask; + t.Dispose(); + added = true; + break; + } + } + + if (added) continue; + + // If AOF sync task did not exist and was not added we added below + // Check if array can hold a new AOF sync task + if (numTasks == tasks.Length) + { + var old_tasks = tasks; + var _tasks = new AofSyncTaskInfo[tasks.Length * 2]; + Array.Copy(tasks, _tasks, tasks.Length); + tasks = _tasks; + Array.Clear(old_tasks); + } + // Add new AOF sync task + tasks[numTasks++] = rss.GetAofSyncTask; + } + + success = true; + } + finally + { + _lock.WriteUnlock(); + + if (!success) + { + foreach (var rss in replicaSyncSessions) + { + if (rss == null) continue; + if (rss.GetAofSyncTask != null) + { + rss.GetAofSyncTask.Dispose(); + } + } + } + } + + return true; + } + public bool TryRemove(AofSyncTaskInfo aofSyncTask) { // Lock addition of new tasks diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs index e4fe2e8191..682433cf14 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs @@ -48,17 +48,47 @@ public void Dispose() } } - public bool TryAddReplicaSyncSession(string remoteNodeId, string remote_primary_replid, CheckpointEntry remoteEntry, long replicaAofBeginAddress, long replicaAofTailAddress) + public bool TryAddReplicaSyncSession(ReplicaSyncSession session) { - var retSession = new ReplicaSyncSession(storeWrapper, clusterProvider, remoteNodeId, remote_primary_replid, remoteEntry, replicaAofBeginAddress, replicaAofTailAddress, logger); - bool success = false; try { _lock.WriteLock(); - for (int i = 0; i < numSessions; i++) + for (var i = 0; i < numSessions; i++) + { + var s = sessions[i]; + if (s.replicaNodeId == session.replicaSyncMetadata.originNodeId) + { + logger?.LogError("Error syncSession for {replicaNodeId} already exists", session.replicaNodeId); + return false; + } + } + + GrowSessionArray(); + sessions[numSessions++] = session; + return true; + } + catch (Exception ex) + { + logger?.LogError(ex, $"{nameof(TryAddReplicaSyncSession)}"); + return false; + } + finally + { + _lock.WriteUnlock(); + } + } + + public bool TryAddReplicaSyncSession(string replicaNodeId, string replicaAssignedPrimaryId, CheckpointEntry replicaCheckpointEntry, long replicaAofBeginAddress, long replicaAofTailAddress) + { + var retSession = new ReplicaSyncSession(storeWrapper, clusterProvider, replicaSyncMetadata: null, replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress, logger); + var success = false; + try + { + _lock.WriteLock(); + for (var i = 0; i < numSessions; i++) { var s = sessions[i]; - if (s.remoteNodeId == retSession.remoteNodeId) + if (s.replicaNodeId == retSession.replicaNodeId) { success = false; return false; @@ -103,7 +133,7 @@ public bool TryRemove(string remoteNodeId) for (int i = 0; i < numSessions; i++) { var s = sessions[i]; - if (s.remoteNodeId == remoteNodeId) + if (s.replicaNodeId == remoteNodeId) { sessions[i] = null; if (i < numSessions - 1) @@ -150,7 +180,7 @@ public bool TryGetSession(string remoteNodeId, out ReplicaSyncSession session) for (int i = 0; i < numSessions; i++) { session = sessions[i]; - if (session.remoteNodeId == remoteNodeId) + if (session.replicaNodeId == remoteNodeId) return true; } return false; @@ -160,5 +190,32 @@ public bool TryGetSession(string remoteNodeId, out ReplicaSyncSession session) _lock.ReadUnlock(); } } + + public ReplicaSyncSession[] GetSessions() => sessions; + + public int GetNumSessions() => numSessions; + + /// + /// Clear references to entries + /// + /// + public void Clear() + { + try + { + _lock.WriteLock(); + if (_disposed) return; + for (var i = 0; i < numSessions; i++) + { + var s = sessions[i]; + sessions[i] = null; + } + numSessions = 0; + } + finally + { + _lock.WriteUnlock(); + } + } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs index f7a898ea84..98cff0c9eb 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicationPrimaryAofSync.cs @@ -23,6 +23,9 @@ internal sealed partial class ReplicationManager : IDisposable public bool TryAddReplicationTask(string nodeid, long startAddress, out AofSyncTaskInfo aofSyncTaskInfo) => aofTaskStore.TryAddReplicationTask(nodeid, startAddress, out aofSyncTaskInfo); + public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, long startAddress) + => aofTaskStore.TryAddReplicationTasks(replicaSyncSessions, startAddress); + public long AofTruncatedUntil => aofTaskStore.AofTruncatedUntil; public bool TryRemoveReplicationTask(AofSyncTaskInfo aofSyncTaskInfo) From ea8f8bbfe5c736a14ac7639638fcf32b9e605b9a Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:52:37 -0800 Subject: [PATCH 04/32] core diskless replication implementation --- ...arnetClientSessionReplicationExtensions.cs | 114 ++++++++ .../DisklessReplication/ReplicaSyncSession.cs | 272 ++++++++++++++++++ .../ReplicationSnapshotIterator.cs | 212 ++++++++++++++ .../ReplicationSyncManager.cs | 214 ++++++++++++++ .../DisklessReplication/SyncStatus.cs | 19 ++ .../PrimaryOps/PrimarySendCheckpoint.cs | 61 ---- .../Replication/PrimaryOps/PrimarySync.cs | 120 ++++++++ .../ReplicaOps/ReplicaDisklessSync.cs | 167 +++++++++++ .../Server/Replication/ReplicationManager.cs | 3 + .../Server/Replication/SyncMetadata.cs | 89 ++++++ libs/server/AOF/AofProcessor.cs | 14 +- 11 files changed, 1222 insertions(+), 63 deletions(-) create mode 100644 libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs create mode 100644 libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs create mode 100644 libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs create mode 100644 libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs delete mode 100644 libs/cluster/Server/Replication/PrimaryOps/PrimarySendCheckpoint.cs create mode 100644 libs/cluster/Server/Replication/PrimaryOps/PrimarySync.cs create mode 100644 libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs create mode 100644 libs/cluster/Server/Replication/SyncMetadata.cs diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs index 6e7b8a748c..25115d7e76 100644 --- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs @@ -19,6 +19,8 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo static ReadOnlySpan send_ckpt_metadata => "SEND_CKPT_METADATA"u8; static ReadOnlySpan send_ckpt_file_segment => "SEND_CKPT_FILE_SEGMENT"u8; static ReadOnlySpan begin_replica_recover => "BEGIN_REPLICA_RECOVER"u8; + static ReadOnlySpan attach_sync => "ATTACH_SYNC"u8; + static ReadOnlySpan sync => "SYNC"u8; /// /// Initiate checkpoint retrieval from replica by sending replica checkpoint information and AOF address range @@ -352,5 +354,117 @@ public Task ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se Interlocked.Increment(ref numCommands); return tcs.Task; } + + /// + /// Initiate attach from replica + /// + /// + /// + public Task ExecuteAttachSync(byte[] syncMetadata) + { + var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + tcsQueue.Enqueue(tcs); + byte* curr = 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(attach_sync, ref curr, end)) + { + Flush(); + curr = offset; + } + offset = curr; + + //3 + while (!RespWriteUtils.TryWriteBulkString(syncMetadata, ref curr, end)) + { + Flush(); + curr = offset; + } + offset = curr; + + Flush(); + Interlocked.Increment(ref numCommands); + return tcs.Task; + } + + /// + /// Set CLUSTER SYNC header info + /// + /// + /// + public void SetClusterSyncHeader(string sourceNodeId, bool isMainStore) + { + currTcsIterationTask = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + tcsQueue.Enqueue(currTcsIterationTask); + curr = offset; + this.isMainStore = isMainStore; + var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE; + + var arraySize = 5; + 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(sync, ref curr, end)) + { + Flush(); + curr = offset; + } + offset = curr; + + // 3 + while (!RespWriteUtils.TryWriteAsciiBulkString(sourceNodeId, ref curr, end)) + { + Flush(); + curr = offset; + } + offset = curr; + + // 4 + while (!RespWriteUtils.TryWriteBulkString(storeType, ref curr, end)) + { + Flush(); + curr = offset; + } + offset = curr; + + // 5 + // Reserve space for the bulk string header + final newline + while (ExtraSpace + 2 > (int)(end - curr)) + { + Flush(); + curr = offset; + } + head = curr; + curr += ExtraSpace; + } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs new file mode 100644 index 0000000000..74a1fca75c --- /dev/null +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -0,0 +1,272 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Garnet.common; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.cluster +{ + internal sealed partial class ReplicaSyncSession + { + SyncStatusInfo ssInfo; + Task flushTask; + AofSyncTaskInfo aofSyncTask = null; + + bool sendMainStore = false; + bool sendObjectStore = false; + bool truncatedAof = false; + bool fullSync = false; + + public bool IsConnected => aofSyncTask != null && aofSyncTask.IsConnected; + + public bool Failed => ssInfo.syncStatus == SyncStatus.FAILED; + + public SyncStatusInfo GetSyncStatusInfo => ssInfo; + + public long currentStoreVersion; + + public long currentObjectStoreVersion; + + /// + /// LogError + /// + public void LogError() + { + logger?.LogError("{msg} > " + + "originNodeId: {originNodeId}, " + + "currentPrimaryReplId: {currentPrimaryReplId}, " + + "currentAofBeginAddress: {currentAofBeginAddress}, " + + "currentAofTailAddress: {currentAofTailAddress}, ", + ssInfo.error, + replicaSyncMetadata.originNodeId, + replicaSyncMetadata.currentPrimaryReplId, + replicaSyncMetadata.currentAofBeginAddress, + replicaSyncMetadata.currentAofTailAddress); + } + + #region NetworkMethods + /// + /// Connect client + /// + public void Connect() + { + if (!aofSyncTask.IsConnected) + aofSyncTask.garnetClient.Connect(); + } + + /// + /// Execute async command + /// + /// + /// + public Task ExecuteAsync(params string[] commands) + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + return aofSyncTask.garnetClient.ExecuteAsync(commands); + } + + /// + /// Set Cluster Sync header + /// + /// + public void SetClysterSyncHeader(bool isMainStore) + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + if (aofSyncTask.garnetClient.NeedsInitialization) + aofSyncTask.garnetClient.SetClusterSyncHeader(clusterProvider.clusterManager.CurrentConfig.LocalNodeId, isMainStore: isMainStore); + } + + /// + /// Try write main store key value pair + /// + /// + /// + /// + /// + public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out Task task) + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + return aofSyncTask.garnetClient.TryWriteKeyValueSpanByte(ref key, ref value, out task); + } + + /// + /// Try write object store key value pair + /// + /// + /// + /// + /// + /// + public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, out Task task) + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + return aofSyncTask.garnetClient.TryWriteKeyValueByteArray(key, value, expiration, out task); + } + + /// + /// Send and reset iteration buffer + /// + /// + public void SendAndResetIterationBuffer(TimeSpan timeout, CancellationToken token) + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + SetFlushTask(aofSyncTask.garnetClient.SendAndResetIterationBuffer(), timeout: timeout, token: token); + } + #endregion + + /// + /// Associated aof sync task instance with this replica sync session + /// + /// + public void AddAofSyncTask(AofSyncTaskInfo aofSyncTask) => this.aofSyncTask = aofSyncTask; + + /// + /// Get the associated aof sync task instance with this replica sync session + /// + public AofSyncTaskInfo GetAofSyncTask => aofSyncTask; + + /// + /// Wait until sync of checkpoint is completed + /// + /// + public async Task CompletePending() + { + while (ssInfo.syncStatus == SyncStatus.INPROGRESS) + await Task.Yield(); + } + + /// + /// Should stream + /// + /// + public bool ShouldStreamCheckpoint() + { + // TODO: implement disk-based logic if possible + return clusterProvider.serverOptions.ReplicaDisklessSync ? + ShouldStreamDisklessCheckpoint() : true; + + bool ShouldStreamDisklessCheckpoint() + { + var localPrimaryReplId = clusterProvider.replicationManager.PrimaryReplId; + var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal); + sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion; + sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion; + truncatedAof = replicaSyncMetadata.currentAofTailAddress < aofSyncTask.StartAddress; + + // 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 + fullSync = sendMainStore || sendObjectStore || truncatedAof; + return fullSync; + } + } + + /// + /// Set status of replica sync session + /// + /// + /// + public void SetStatus(SyncStatus status, string error = null) + { + ssInfo.error = error; + // NOTE: set this last to signal state change + ssInfo.syncStatus = status; + } + + /// + /// Set network flush task for checkpoint snapshot stream data + /// + /// + /// + /// + public void SetFlushTask(Task task, TimeSpan timeout, CancellationToken token) + { + flushTask = task.ContinueWith(resp => + { + if (!resp.Result.Equals("OK", StringComparison.Ordinal)) + { + logger?.LogError("ReplicaSyncSession: {errorMsg}", resp.Result); + SetStatus(SyncStatus.FAILED, resp.Result); + return false; + } + return true; + }, TaskContinuationOptions.OnlyOnRanToCompletion).WaitAsync(timeout, token); + } + + /// + /// Wait for network buffer flush + /// + /// + public async Task WaitForFlush() + { + try + { + _ = await flushTask; + flushTask = null; + } + catch (Exception ex) + { + logger?.LogError(ex, "{method}", $"{nameof(ReplicaSyncSession.WaitForFlush)}"); + SetStatus(SyncStatus.FAILED, "Flush task faulted"); + } + } + + /// + /// Begin syncing AOF to the replica + /// + public async Task BeginAofSync() + { + var mmr = clusterProvider.serverOptions.MainMemoryReplication; + var aofNull = clusterProvider.serverOptions.UseAofNullDevice; + + var canReplayFromAddress = aofSyncTask.StartAddress; + var replicaAofBeginAddress = replicaSyncMetadata.currentAofBeginAddress; + var replicaAofTailAddress = replicaSyncMetadata.currentAofTailAddress; + + var currentAofBeginAddress = canReplayFromAddress; + var currentTailAofAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; + + // TODO: + // If partial sync we need to calculate the beginAddress and endAddress of replica AOF + // to match the primary AOF + if (!fullSync) + { + + } + + var recoverSyncMetadata = new SyncMetadata( + fullSync: fullSync, + originNodeRole: clusterProvider.clusterManager.CurrentConfig.LocalNodeRole, + originNodeId: clusterProvider.clusterManager.CurrentConfig.LocalNodeId, + currentPrimaryReplId: clusterProvider.replicationManager.PrimaryReplId, + currentStoreVersion: currentStoreVersion, + currentObjectStoreVersion: currentObjectStoreVersion, + currentAofBeginAddress: currentAofBeginAddress, + currentAofTailAddress: currentTailAofAddress, + currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset, + checkpointEntry: null); + + var result = await aofSyncTask.garnetClient.ExecuteAttachSync(recoverSyncMetadata.ToByteArray()); + if (!long.TryParse(result, out var syncFromAofAddress)) + { + logger?.LogError("Failed to parse syncFromAddress at {method}", nameof(BeginAofSync)); + SetStatus(SyncStatus.FAILED, "Failed to parse recovery offset"); + return; + } + + // We have already added the iterator for the covered address above but replica might request an address + // that is ahead of the covered address so we should start streaming from that address in order not to + // introduce duplicate insertions. + if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaSyncMetadata.originNodeId, syncFromAofAddress, out var aofSyncTaskInfo)) + throw new GarnetException("Failed trying to try update replication task"); + if (!clusterProvider.replicationManager.TryConnectToReplica(replicaSyncMetadata.originNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) + throw new GarnetException("Failed connecting to replica for aofSync"); + } + } +} \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs new file mode 100644 index 0000000000..49d6a36309 --- /dev/null +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -0,0 +1,212 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using Garnet.server; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +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; + + readonly ReplicaSyncSession[] sessions; + readonly int numSessions; + + public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, TimeSpan timeout, CancellationToken cancellationToken, ILogger logger = null) + { + this.replicationSyncManager = replicationSyncManager; + this.timeout = timeout; + this.cancellationToken = cancellationToken; + this.logger = logger; + + var sessionStore = replicationSyncManager.GetSessionStore; + sessions = sessionStore.GetSessions(); + numSessions = sessionStore.GetNumSessions(); + + mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); + if (!replicationSyncManager.GetClusterProvider.serverOptions.DisableObjects) + objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this); + } + + public static bool IsActive(ReplicaSyncSession[] session, int offset) + { + // Check if session is null if an error occurred earlier and session was broken + if (session[offset] == null) + return false; + + // Check if connection is still healthy + if (!session[offset].IsConnected) + { + session[offset].SetStatus(SyncStatus.FAILED, "Connection broken"); + session[offset] = null; + return false; + } + return true; + } + + public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion, bool isMainStore) + { + if (cancellationToken.IsCancellationRequested) + return false; + + for (var i = 0; i < numSessions; i++) + { + if (!IsActive(sessions, i)) continue; + sessions[i].SetClysterSyncHeader(isMainStore: isMainStore); + if (isMainStore) + sessions[i].currentStoreVersion = targetVersion; + else + sessions[i].currentObjectStoreVersion = targetVersion; + } + + return true; + } + + void WaitForFlushAll() + { + // Wait for flush to complete for all and retry to enqueue previous keyValuePair above + for (var i = 0; i < numSessions; i++) + { + if (!IsActive(sessions, i)) continue; + sessions[i].WaitForFlush().GetAwaiter().GetResult(); + } + } + + public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords) + { + var needToFlush = false; + while (true) + { + if (cancellationToken.IsCancellationRequested) + return false; + + // Write key value pair to network buffer + for (var i = 0; i < numSessions; i++) + { + if (!IsActive(sessions, i)) continue; + + // Initialize header if necessary + sessions[i].SetClysterSyncHeader(isMainStore: true); + + // Try to write to network buffer. If failed we need to retry + if (!sessions[i].TryWriteKeyValueSpanByte(ref key, ref value, out var task)) + { + sessions[i].SetFlushTask(task, timeout, cancellationToken); + needToFlush = true; + } + } + + if (!needToFlush) break; + + // Wait for flush to complete for all and retry to enqueue previous keyValuePair above + WaitForFlushAll(); + } + + return true; + } + + public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords) + { + var needToFlush = false; + var objectData = GarnetObjectSerializer.Serialize(value); + while (true) + { + if (cancellationToken.IsCancellationRequested) + return false; + + // Write key value pair to network buffer + for (var i = 0; i < numSessions; i++) + { + if (!IsActive(sessions, i)) continue; + + // Initialize header if necessary + sessions[i].SetClysterSyncHeader(isMainStore: false); + + // Try to write to network buffer. If failed we need to retry + if (!sessions[i].TryWriteKeyValueByteArray(key, objectData, value.Expiration, out var task)) + { + sessions[i].SetFlushTask(task, timeout, cancellationToken); + needToFlush = true; + } + } + + if (!needToFlush) break; + + // Wait for flush to complete for all and retry to enqueue previous keyValuePair above + WaitForFlushAll(); + } + + return true; + } + + public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long targetVersion) + { + // Flush remaining data + for (var i = 0; i < numSessions; i++) + { + if (!IsActive(sessions, i)) continue; + sessions[i].SendAndResetIterationBuffer(timeout, cancellationToken); + } + + // Wait for flush and response to complete + WaitForFlushAll(); + + // Enqueue version change commit + replicationSyncManager.GetClusterProvider.storeWrapper.EnqueueCommit(isMainStore, targetVersion); + } + } + + internal sealed unsafe class MainStoreSnapshotIterator(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); + } + + public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords) + => snapshotIteratorManager.Reader(ref key, ref value, recordMetadata, numberOfRecords); + + public void OnException(Exception exception, long numberOfRecords) + => snapshotIteratorManager.logger?.LogError(exception, $"{nameof(MainStoreSnapshotIterator)}"); + + 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(ref byte[] key, ref IGarnetObject value, RecordMetadata recordMetadata, long numberOfRecords) + => snapshotIteratorManager.Reader(ref key, ref value, 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 new file mode 100644 index 0000000000..b6383e3a30 --- /dev/null +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -0,0 +1,214 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; +using Garnet.common; +using Microsoft.Extensions.Logging; +using Tsavorite.core; + +namespace Garnet.cluster +{ + internal sealed class ReplicationSyncManager + { + SingleWriterMultiReaderLock syncInProgress; + readonly CancellationTokenSource cts; + readonly TimeSpan clusterTimeout; + readonly ILogger logger; + + public ReplicaSyncSessionTaskStore GetSessionStore { get; } + + public ClusterProvider GetClusterProvider { get; } + + public ReplicationSyncManager(ClusterProvider clusterProvider, ILogger logger = null) + { + GetSessionStore = new ReplicaSyncSessionTaskStore(clusterProvider.storeWrapper, clusterProvider, logger); + this.GetClusterProvider = clusterProvider; + this.logger = logger; + + var opts = clusterProvider.serverOptions; + clusterTimeout = opts.ClusterTimeout <= 0 ? Timeout.InfiniteTimeSpan : TimeSpan.FromSeconds(opts.ClusterTimeout); + cts = new(); + } + + public void Dispose() + { + cts.Cancel(); + cts.Dispose(); + syncInProgress.WriteLock(); + } + + /// + /// Begin background replica sync session + /// + /// Replica sync metadata + /// Replica sync session created + /// + public bool AddSyncSession(SyncMetadata replicaSyncMetadata, out ReplicaSyncSession replicaSyncSession) + { + replicaSyncSession = new ReplicaSyncSession(GetClusterProvider.storeWrapper, GetClusterProvider, replicaSyncMetadata, logger: logger); + replicaSyncSession.SetStatus(SyncStatus.INITIALIZING); + try + { + syncInProgress.ReadLock(); + return GetSessionStore.TryAddReplicaSyncSession(replicaSyncSession); + } + finally + { + syncInProgress.ReadUnlock(); + } + } + + /// + /// Start sync session + /// + /// + /// + public async Task MainDisklessSync(ReplicaSyncSession replicaSyncSession) + { + // Give opportunity to other replicas to attach for streaming sync + if (GetClusterProvider.serverOptions.ReplicaDisklessSyncDelay > 0) + Thread.Sleep(TimeSpan.FromSeconds(GetClusterProvider.serverOptions.ReplicaDisklessSyncDelay)); + + // Started syncing + replicaSyncSession.SetStatus(SyncStatus.INPROGRESS); + + // Only one thread will acquire this lock + if (syncInProgress.OneWriteLock()) + { + // Launch a background task to sync the attached replicas using streaming snapshot + _ = Task.Run(() => StreamingSnapshotSync()); + } + + // Wait for main sync task to complete + await replicaSyncSession.CompletePending(); + + // If session faulted return early + if (replicaSyncSession.Failed) + { + replicaSyncSession.LogError(); + replicaSyncSession.Dispose(); + return replicaSyncSession.GetSyncStatusInfo; + } + + await replicaSyncSession.BeginAofSync(); + + return replicaSyncSession.GetSyncStatusInfo; + } + + // Main streaming snapshot task + async Task StreamingSnapshotSync() + { + // Parameters for sync operation + var disklessRepl = GetClusterProvider.serverOptions.ReplicaDisklessSync; + var disableObjects = GetClusterProvider.serverOptions.DisableObjects; + + // Replica sync session + var numSessions = GetSessionStore.GetNumSessions(); + var sessions = GetSessionStore.GetSessions(); + + try + { + // Take lock to ensure no other task will be taking a checkpoint + while (!GetClusterProvider.storeWrapper.TryPauseCheckpoints()) + await Task.Yield(); + + // Get sync metadata for checkpoint + await PrepareForSync(); + + // Stream checkpoint to replicas + await StreamDisklessCheckpoint(); + } + finally + { + // Notify sync session of success success + for (var i = 0; i < numSessions; i++) + sessions[i]?.SetStatus(SyncStatus.SUCCESS); + + // Clear array of sync sessions + GetSessionStore.Clear(); + + // Release checkpoint lock + GetClusterProvider.storeWrapper.ResumeCheckpoints(); + + // Unlock sync session lock + syncInProgress.WriteUnlock(); + } + + // Acquire checkpoint and lock AOF if possible + async Task PrepareForSync() + { + if (disklessRepl) + { + #region pauseAofTruncation + while (true) + { + // Calculate minimum address from which replicas should start streaming from + var syncFromAddress = GetClusterProvider.storeWrapper.appendOnlyFile.TailAddress; + + // Lock AOF address for sync streaming + if (GetClusterProvider.replicationManager.TryAddReplicationTasks(GetSessionStore.GetSessions(), syncFromAddress)) + break; + + // Retry if failed to lock AOF address because truncation occurred + await Task.Yield(); + } + #endregion + + #region initializeConnection + for (var i = 0; i < numSessions; i++) + { + try + { + // Initialize connections + sessions[i].Connect(); + + // Set store version to operate on + sessions[i].currentStoreVersion = GetClusterProvider.storeWrapper.store.CurrentVersion; + sessions[i].currentObjectStoreVersion = disableObjects ? -1 : GetClusterProvider.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 + if (!sessions[i].ShouldStreamCheckpoint()) + { + sessions[i]?.SetStatus(SyncStatus.SUCCESS, "Partial sync"); + sessions[i] = null; + } + else + { + // Reset replica database in preparation for full sync + sessions[i].SetFlushTask(sessions[i].ExecuteAsync(["FLUSHALL"]), timeout: clusterTimeout, cts.Token); + await sessions[i].WaitForFlush(); + if (sessions[i].Failed) sessions[i] = null; + } + } + catch (Exception ex) + { + sessions[i]?.SetStatus(SyncStatus.FAILED, ex.Message); + sessions[i] = null; + } + + } + #endregion + } + } + } + + async Task StreamDisklessCheckpoint() + { + // Main snapshot iterator manager + var manager = new SnapshotIteratorManager(this, clusterTimeout, cts.Token, logger); + + // Iterate through main store + var mainStoreResult = await GetClusterProvider.storeWrapper.store. + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + + if (!GetClusterProvider.serverOptions.DisableObjects) + { + // Iterate through object store + var objectStoreResult = await GetClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + } + } + } +} \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs new file mode 100644 index 0000000000..e828e1c78a --- /dev/null +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs @@ -0,0 +1,19 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Garnet.cluster +{ + enum SyncStatus : byte + { + SUCCESS, + FAILED, + INPROGRESS, + INITIALIZING + } + + struct SyncStatusInfo + { + public SyncStatus syncStatus; + public string error; + } +} diff --git a/libs/cluster/Server/Replication/PrimaryOps/PrimarySendCheckpoint.cs b/libs/cluster/Server/Replication/PrimaryOps/PrimarySendCheckpoint.cs deleted file mode 100644 index 11ba423ea3..0000000000 --- a/libs/cluster/Server/Replication/PrimaryOps/PrimarySendCheckpoint.cs +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Text; -using Microsoft.Extensions.Logging; - -namespace Garnet.cluster -{ - internal sealed partial class ReplicationManager : IDisposable - { - readonly ReplicaSyncSessionTaskStore replicaSyncSessionTaskStore; - - /// - /// Begin background replica sync session - /// - /// The ASCII encoded error message if the method returned ; otherwise - public bool TryBeginReplicaSyncSession(string remoteNodeId, string remote_primary_replid, CheckpointEntry remoteEntry, long replicaAofBeginAddress, long replicaAofTailAddress, out ReadOnlySpan errorMessage) - { - errorMessage = default; - if (!replicaSyncSessionTaskStore.TryAddReplicaSyncSession(remoteNodeId, remote_primary_replid, remoteEntry, replicaAofBeginAddress, replicaAofTailAddress)) - { - errorMessage = CmdStrings.RESP_ERR_CREATE_SYNC_SESSION_ERROR; - logger?.LogError("{errorMessage}", Encoding.ASCII.GetString(errorMessage)); - return false; - } - - if (!ReplicaSyncSessionBackgroundTask(remoteNodeId, out errorMessage)) - { - return false; - } - return true; - } - - private bool ReplicaSyncSessionBackgroundTask(string replicaId, out ReadOnlySpan errorMessage) - { - try - { - if (!replicaSyncSessionTaskStore.TryGetSession(replicaId, out var session)) - { - errorMessage = CmdStrings.RESP_ERR_RETRIEVE_SYNC_SESSION_ERROR; - logger?.LogError("{errorMessage}", Encoding.ASCII.GetString(errorMessage)); - return false; - } - - if (!session.SendCheckpoint().GetAwaiter().GetResult()) - { - errorMessage = Encoding.ASCII.GetBytes(session.errorMsg); - return false; - } - - errorMessage = CmdStrings.RESP_OK; - return true; - } - finally - { - replicaSyncSessionTaskStore.TryRemove(replicaId); - } - } - } -} \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/PrimarySync.cs b/libs/cluster/Server/Replication/PrimaryOps/PrimarySync.cs new file mode 100644 index 0000000000..5fd3f2b5ab --- /dev/null +++ b/libs/cluster/Server/Replication/PrimaryOps/PrimarySync.cs @@ -0,0 +1,120 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Text; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; + +namespace Garnet.cluster +{ + internal sealed partial class ReplicationManager : IDisposable + { + readonly ReplicaSyncSessionTaskStore replicaSyncSessionTaskStore; + + /// + /// Try attach sync session from replica + /// + /// + /// + /// + public bool TryAttachSync(SyncMetadata replicaSyncMetadata, out ReadOnlySpan errorMessage) + { + errorMessage = null; + if (clusterProvider.serverOptions.ReplicaDisklessSync) + { + var status = TryAttachDisklessSync(replicaSyncMetadata).GetAwaiter().GetResult(); + if (status.syncStatus == SyncStatus.FAILED) + errorMessage = Encoding.ASCII.GetBytes(status.error); + } + + return true; + + async Task TryAttachDisklessSync(SyncMetadata replicaSyncMetadata) + { + if (!replicationSyncManager.AddSyncSession(replicaSyncMetadata, out var replicaSyncSession)) + { + replicaSyncSession?.Dispose(); + return new() { syncStatus = SyncStatus.FAILED, error = "failed to add sync session" }; + } + + return await replicationSyncManager.MainDisklessSync(replicaSyncSession); + } + } + + /// + /// Start sync of remote replica from this primary + /// + /// + /// + /// + /// + /// + /// + /// + public bool TryBeginPrimarySync( + string replicaNodeId, + string replicaAssignedPrimaryId, + CheckpointEntry replicaCheckpointEntry, + long replicaAofBeginAddress, + long replicaAofTailAddress, + out ReadOnlySpan errorMessage) + { + return TryBeginDiskSync(replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress, out errorMessage); + } + + /// + /// Begin background replica sync session + /// + /// Node-id of replica that is currently attaching + /// Primary-id of replica that is currently attaching + /// Most recent checkpoint entry at replica + /// AOF begin address at replica + /// AOF tail address at replica + /// The ASCII encoded error message if the method returned ; otherwise + /// + public bool TryBeginDiskSync( + string replicaNodeId, + string replicaAssignedPrimaryId, + CheckpointEntry replicaCheckpointEntry, + long replicaAofBeginAddress, + long replicaAofTailAddress, + out ReadOnlySpan errorMessage) + { + if (!replicaSyncSessionTaskStore.TryAddReplicaSyncSession(replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress)) + { + errorMessage = CmdStrings.RESP_ERR_CREATE_SYNC_SESSION_ERROR; + logger?.LogError("{errorMessage}", Encoding.ASCII.GetString(errorMessage)); + return false; + } + + return ReplicaSyncSessionBackgroundTask(replicaNodeId, out errorMessage); + + bool ReplicaSyncSessionBackgroundTask(string replicaId, out ReadOnlySpan errorMessage) + { + try + { + if (!replicaSyncSessionTaskStore.TryGetSession(replicaId, out var session)) + { + errorMessage = CmdStrings.RESP_ERR_RETRIEVE_SYNC_SESSION_ERROR; + logger?.LogError("{errorMessage}", Encoding.ASCII.GetString(errorMessage)); + return false; + } + + if (!session.SendCheckpoint().GetAwaiter().GetResult()) + { + errorMessage = Encoding.ASCII.GetBytes(session.errorMsg); + return false; + } + + errorMessage = CmdStrings.RESP_OK; + return true; + } + finally + { + replicaSyncSessionTaskStore.TryRemove(replicaId); + } + } + } + } +} \ No newline at end of file diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs new file mode 100644 index 0000000000..9895ca4622 --- /dev/null +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -0,0 +1,167 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Text; +using System.Threading.Tasks; +using Garnet.client; +using Microsoft.Extensions.Logging; + +namespace Garnet.cluster +{ + internal sealed partial class ReplicationManager : IDisposable + { + public bool TryReplicateDisklessSync( + ClusterSession session, + string nodeId, + bool background, + bool force, + out ReadOnlySpan errorMessage) + { + errorMessage = default; + // Ensure two replicate commands do not execute at the same time. + if (!replicateLock.TryWriteLock()) + { + errorMessage = CmdStrings.RESP_ERR_GENERIC_CANNOT_ACQUIRE_REPLICATE_LOCK; + return false; + } + + try + { + logger?.LogTrace("CLUSTER REPLICATE {nodeid}", nodeId); + if (!clusterProvider.clusterManager.TryAddReplica(nodeId, force: force, out errorMessage, logger: logger)) + { + replicateLock.WriteUnlock(); + return false; + } + + // Wait for threads to agree configuration change of this node + session.UnsafeBumpAndWaitForEpochTransition(); + + _ = Task.Run(() => TryBeginReplicaSync()); + } + catch (Exception ex) + { + logger?.LogError(ex, $"{nameof(TryReplicateDisklessSync)}"); + replicateLock.WriteUnlock(); + } + + async Task TryBeginReplicaSync() + { + var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync; + var dissableObjects = clusterProvider.serverOptions.DisableObjects; + try + { + if (!clusterProvider.serverOptions.EnableFastCommit) + { + storeWrapper.appendOnlyFile?.Commit(); + storeWrapper.appendOnlyFile?.WaitForCommit(); + } + + // Reset background replay iterator + ResetReplayIterator(); + + // Reset the database in preparation for connecting to primary + // only if we expect to have disk checkpoint to recover from, + // otherwise the replica will receive a reset message from primary if needed + if (!disklessSync) + storeWrapper.Reset(); + + // Send request to primary + // Primary will initiate background task and start sending checkpoint data + // + // Replica waits for retrieval to complete before moving forward to recovery + // Retrieval completion coordinated by remoteCheckpointRetrievalCompleted + var current = clusterProvider.clusterManager.CurrentConfig; + var (address, port) = current.GetLocalNodePrimaryAddress(); + GarnetClientSession gcs = null; + CheckpointEntry checkpointEntry = null; + + if (!disklessSync) + checkpointEntry = GetLatestCheckpointEntryFromDisk(); + + if (address == null || port == -1) + { + var errorMsg = Encoding.ASCII.GetString(CmdStrings.RESP_ERR_GENERIC_NOT_ASSIGNED_PRIMARY_ERROR); + logger?.LogError("{msg}", errorMsg); + return; + } + + gcs = new( + address, + port, + clusterProvider.replicationManager.GetIRSNetworkBufferSettings, + clusterProvider.replicationManager.GetNetworkPool, + tlsOptions: clusterProvider.serverOptions.TlsOptions?.TlsClientOptions, + authUsername: clusterProvider.ClusterUsername, + authPassword: clusterProvider.ClusterPassword); + + // Used only for disk-based replication + if (!disklessSync) + recvCheckpointHandler = new ReceiveCheckpointHandler(clusterProvider, logger); + gcs.Connect(); + + SyncMetadata syncMetadata = new( + fullSync: false, + originNodeRole: current.LocalNodeRole, + originNodeId: current.LocalNodeId, + currentPrimaryReplId: PrimaryReplId, + currentStoreVersion: storeWrapper.store.CurrentVersion, + currentObjectStoreVersion: dissableObjects ? -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); + } + catch (Exception ex) + { + logger?.LogError(ex, $"{nameof(TryBeginReplicaSync)}"); + clusterProvider.clusterManager.TryResetReplica(); + SuspendRecovery(); + } + finally + { + recvCheckpointHandler?.Dispose(); + } + } + + return true; + } + + public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata) + { + try + { + var aofBeginAddress = primarySyncMetadata.currentAofBeginAddress; + var aofTailAddress = aofBeginAddress; + var replicationOffset = aofBeginAddress; + + if (!primarySyncMetadata.fullSync) + { + // TODO: replay local AOF + } + + storeWrapper.appendOnlyFile.Initialize(aofBeginAddress, aofBeginAddress); + + // Set DB version + storeWrapper.store.SetVersion(primarySyncMetadata.currentStoreVersion); + if (!clusterProvider.serverOptions.DisableObjects) + storeWrapper.objectStore.SetVersion(primarySyncMetadata.currentObjectStoreVersion); + + // Update replicationId to mark any subsequent checkpoints as part of this history + logger?.LogInformation("Updating ReplicationId"); + TryUpdateMyPrimaryReplId(primarySyncMetadata.currentPrimaryReplId); + + ReplicationOffset = replicationOffset; + return ReplicationOffset; + } + finally + { + // Done with recovery at this point + SuspendRecovery(); + } + } + } +} diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index 64819ca9c3..ac10722efa 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -19,6 +19,8 @@ internal sealed partial class ReplicationManager : IDisposable readonly StoreWrapper storeWrapper; readonly AofProcessor aofProcessor; readonly CheckpointStore checkpointStore; + readonly ReplicationSyncManager replicationSyncManager; + readonly CancellationTokenSource ctsRepManager = new(); @@ -101,6 +103,7 @@ public ReplicationManager(ClusterProvider clusterProvider, ILogger logger = null aofProcessor = new AofProcessor(storeWrapper, recordToAof: false, logger: logger); replicaSyncSessionTaskStore = new ReplicaSyncSessionTaskStore(storeWrapper, clusterProvider, logger); + replicationSyncManager = new ReplicationSyncManager(clusterProvider, logger); ReplicationOffset = 0; diff --git a/libs/cluster/Server/Replication/SyncMetadata.cs b/libs/cluster/Server/Replication/SyncMetadata.cs new file mode 100644 index 0000000000..2b7e31d658 --- /dev/null +++ b/libs/cluster/Server/Replication/SyncMetadata.cs @@ -0,0 +1,89 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.IO; +using System.Text; + +namespace Garnet.cluster +{ + sealed class SyncMetadata( + bool fullSync, + NodeRole originNodeRole, + string originNodeId, + string currentPrimaryReplId, + long currentStoreVersion, + long currentObjectStoreVersion, + long currentAofBeginAddress, + long currentAofTailAddress, + long currentReplicationOffset, + CheckpointEntry checkpointEntry) + { + public readonly bool fullSync = fullSync; + public readonly NodeRole originNodeRole = originNodeRole; + 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; + public readonly CheckpointEntry checkpointEntry = checkpointEntry; + + public byte[] ToByteArray() + { + var ms = new MemoryStream(); + var writer = new BinaryWriter(ms, Encoding.ASCII); + + writer.Write(fullSync); + writer.Write((byte)originNodeRole); + writer.Write(originNodeId); + writer.Write(currentPrimaryReplId); + + writer.Write(currentStoreVersion); + writer.Write(currentObjectStoreVersion); + + writer.Write(currentAofBeginAddress); + writer.Write(currentAofTailAddress); + writer.Write(currentReplicationOffset); + + if (checkpointEntry != null) + { + var bb = checkpointEntry.ToByteArray(); + writer.Write(bb.Length); + writer.Write(bb); + } + else + { + writer.Write(0); + } + + var byteArray = ms.ToArray(); + writer.Dispose(); + ms.Dispose(); + return byteArray; + } + + public static SyncMetadata FromByteArray(byte[] serialized) + { + var ms = new MemoryStream(serialized); + var reader = new BinaryReader(ms); + var syncMetadata = new SyncMetadata + ( + fullSync: reader.ReadBoolean(), + originNodeRole: (NodeRole)reader.ReadByte(), + originNodeId: reader.ReadString(), + currentPrimaryReplId: reader.ReadString(), + currentStoreVersion: reader.ReadInt64(), + currentObjectStoreVersion: reader.ReadInt64(), + currentAofBeginAddress: reader.ReadInt64(), + currentAofTailAddress: reader.ReadInt64(), + currentReplicationOffset: reader.ReadInt64(), + checkpointEntry: CheckpointEntry.FromByteArray(reader.ReadBytes(reader.ReadInt32())) + ); + + reader.Dispose(); + ms.Dispose(); + return syncMetadata; + } + } +} \ No newline at end of file diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index aed7b5d27f..ad949ba032 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -211,14 +211,24 @@ public unsafe void ProcessAofRecordInternal(byte* ptr, int length, bool asReplic if (asReplica) { if (header.storeVersion > storeWrapper.store.CurrentVersion) - storeWrapper.TakeCheckpoint(false, StoreType.Main, logger); + { + if (storeWrapper.serverOptions.ReplicaDisklessSync) + storeWrapper.store.SetVersion(header.storeVersion); + else + storeWrapper.TakeCheckpoint(false, StoreType.Main, logger); + } } break; case AofEntryType.ObjectStoreCheckpointCommit: if (asReplica) { if (header.storeVersion > storeWrapper.objectStore.CurrentVersion) - storeWrapper.TakeCheckpoint(false, StoreType.Object, logger); + { + if (storeWrapper.serverOptions.ReplicaDisklessSync) + storeWrapper.objectStore.SetVersion(header.storeVersion); + else + storeWrapper.TakeCheckpoint(false, StoreType.Object, logger); + } } break; default: From e3f35edba985822b87854fd159712a92e0c7e1aa Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:55:06 -0800 Subject: [PATCH 05/32] expose diskless replication API --- libs/cluster/Session/ClusterCommands.cs | 2 + .../Session/RespClusterReplicationCommands.cs | 125 ++++++++++++++++-- libs/resources/RespCommandsInfo.json | 16 +++ libs/server/Resp/CmdStrings.cs | 2 + libs/server/Resp/Parser/RespCommand.cs | 14 +- .../CommandInfoUpdater/SupportedCommand.cs | 4 +- 6 files changed, 152 insertions(+), 11 deletions(-) diff --git a/libs/cluster/Session/ClusterCommands.cs b/libs/cluster/Session/ClusterCommands.cs index d8a682519b..9df6208b3d 100644 --- a/libs/cluster/Session/ClusterCommands.cs +++ b/libs/cluster/Session/ClusterCommands.cs @@ -139,6 +139,7 @@ private void ProcessClusterCommands(RespCommand command, out bool invalidParamet RespCommand.CLUSTER_ADDSLOTSRANGE => NetworkClusterAddSlotsRange(out invalidParameters), RespCommand.CLUSTER_AOFSYNC => NetworkClusterAOFSync(out invalidParameters), RespCommand.CLUSTER_APPENDLOG => NetworkClusterAppendLog(out invalidParameters), + RespCommand.CLUSTER_ATTACH_SYNC => NetworkClusterAttachSync(out invalidParameters), RespCommand.CLUSTER_BANLIST => NetworkClusterBanList(out invalidParameters), RespCommand.CLUSTER_BEGIN_REPLICA_RECOVER => NetworkClusterBeginReplicaRecover(out invalidParameters), RespCommand.CLUSTER_BUMPEPOCH => NetworkClusterBumpEpoch(out invalidParameters), @@ -176,6 +177,7 @@ private void ProcessClusterCommands(RespCommand command, out bool invalidParamet RespCommand.CLUSTER_SHARDS => NetworkClusterShards(out invalidParameters), RespCommand.CLUSTER_SLOTS => NetworkClusterSlots(out invalidParameters), RespCommand.CLUSTER_SLOTSTATE => NetworkClusterSlotState(out invalidParameters), + RespCommand.CLUSTER_SYNC => NetworkClusterSync(out invalidParameters), _ => throw new Exception($"Unexpected cluster subcommand: {command}") }; this.sessionMetrics?.incr_total_cluster_commands_processed(); diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index f61328fe5a..507e64f59b 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -6,6 +6,7 @@ using Garnet.common; using Garnet.server; using Microsoft.Extensions.Logging; +using Tsavorite.core; namespace Garnet.cluster { @@ -86,14 +87,18 @@ private bool NetworkClusterReplicate(out bool invalidParameters) } else { - if (!clusterProvider.replicationManager.TryBeginReplicate(this, nodeId, background: background, force: false, out var errorMessage)) + var success = clusterProvider.serverOptions.ReplicaDisklessSync ? + clusterProvider.replicationManager.TryReplicateDisklessSync(this, nodeId, background: background, force: false, out var errorMessage) : + clusterProvider.replicationManager.TryBeginReplicate(this, nodeId, background: background, force: false, out errorMessage); + + if (success) { - while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) SendAndReset(); } else { - while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) + while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) SendAndReset(); } } @@ -212,8 +217,8 @@ private bool NetworkClusterInitiateReplicaSync(out bool invalidParameters) return true; } - var nodeId = parseState.GetString(0); - var primaryReplicaId = parseState.GetString(1); + var replicaNodeId = parseState.GetString(0); + var replicaAssignedPrimaryId = parseState.GetString(1); var checkpointEntryBytes = parseState.GetArgSliceByRef(2).SpanByte.ToByteArray(); if (!parseState.TryGetLong(3, out var replicaAofBeginAddress) || @@ -224,10 +229,15 @@ private bool NetworkClusterInitiateReplicaSync(out bool invalidParameters) return true; } - var remoteEntry = CheckpointEntry.FromByteArray(checkpointEntryBytes); + var replicaCheckpointEntry = CheckpointEntry.FromByteArray(checkpointEntryBytes); - if (!clusterProvider.replicationManager.TryBeginReplicaSyncSession( - nodeId, primaryReplicaId, remoteEntry, replicaAofBeginAddress, replicaAofTailAddress, out var errorMessage)) + if (!clusterProvider.replicationManager.TryBeginPrimarySync( + replicaNodeId, + replicaAssignedPrimaryId, + replicaCheckpointEntry, + replicaAofBeginAddress, + replicaAofTailAddress, + out var errorMessage)) { while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) SendAndReset(); @@ -367,5 +377,104 @@ private bool NetworkClusterBeginReplicaRecover(out bool invalidParameters) return true; } + + /// + /// Implements CLUSTER attach_sync command (only for internode use) + /// + /// + /// + private bool NetworkClusterAttachSync(out bool invalidParameters) + { + invalidParameters = false; + + // Expecting exactly 1 arguments + if (parseState.Count != 1) + { + invalidParameters = true; + return true; + } + + var checkpointEntryBytes = parseState.GetArgSliceByRef(0).SpanByte.ToByteArray(); + var syncMetadata = SyncMetadata.FromByteArray(checkpointEntryBytes); + + ReadOnlySpan errorMessage = default; + long replicationOffset = -1; + if (syncMetadata.originNodeRole == NodeRole.REPLICA) + _ = clusterProvider.replicationManager.TryAttachSync(syncMetadata, out errorMessage); + else + replicationOffset = clusterProvider.replicationManager.ReplicaRecoverDiskless(syncMetadata); + + if (errorMessage != default) + { + while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) + SendAndReset(); + } + else + { + while (!RespWriteUtils.TryWriteInt64(replicationOffset, ref dcurr, dend)) + SendAndReset(); + } + + return true; + } + + /// + /// Implements CLUSTER SYNC + /// + /// + /// + private bool NetworkClusterSync(out bool invalidParameters) + { + invalidParameters = false; + + // Expecting exactly 3 arguments + if (parseState.Count != 3) + { + invalidParameters = true; + return true; + } + + var primaryNodeId = parseState.GetString(0); + var storeTypeSpan = parseState.GetArgSliceByRef(1).ReadOnlySpan; + var payload = parseState.GetArgSliceByRef(2).SpanByte; + var payloadPtr = payload.ToPointer(); + var lastParam = parseState.GetArgSliceByRef(parseState.Count - 1).SpanByte; + var payloadEndPtr = lastParam.ToPointer() + lastParam.Length; + + var keyValuePairCount = *(int*)payloadPtr; + var i = 0; + payloadPtr += 4; + if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("SSTORE"u8)) + { + TrackImportProgress(keyValuePairCount, isMainStore: true, keyValuePairCount == 0); + while (i < keyValuePairCount) + { + ref var key = ref SpanByte.Reinterpret(payloadPtr); + payloadPtr += key.TotalSize; + ref var value = ref SpanByte.Reinterpret(payloadPtr); + payloadPtr += value.TotalSize; + + _ = basicGarnetApi.SET(ref key, ref value); + i++; + } + } + else if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("OSTORE"u8)) + { + while (i < keyValuePairCount) + { + if (!RespReadUtils.TryReadSerializedData(out var key, out var data, out var expiration, ref payloadPtr, payloadEndPtr)) + return false; + + var value = clusterProvider.storeWrapper.GarnetObjectSerializer.Deserialize(data); + value.Expiration = expiration; + _ = basicGarnetApi.SET(key, value); + } + } + + while (!RespWriteUtils.TryWriteDirect(CmdStrings.RESP_OK, ref dcurr, dend)) + SendAndReset(); + + return true; + } } } \ No newline at end of file diff --git a/libs/resources/RespCommandsInfo.json b/libs/resources/RespCommandsInfo.json index 2f884589ee..e087372954 100644 --- a/libs/resources/RespCommandsInfo.json +++ b/libs/resources/RespCommandsInfo.json @@ -601,6 +601,14 @@ "Flags": "Admin, NoMulti, NoScript", "AclCategories": "Admin, Dangerous, Slow, Garnet" }, + { + "Command": "CLUSTER_ATTACH_SYNC", + "Name": "CLUSTER|ATTACH_SYNC", + "IsInternal": true, + "Arity": 3, + "Flags": "Admin, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow, Garnet" + }, { "Command": "CLUSTER_BANLIST", "Name": "CLUSTER|BANLIST", @@ -926,6 +934,14 @@ "Arity": 1, "Flags": "Admin, NoMulti, NoScript", "AclCategories": "Admin, Dangerous, Slow, Garnet" + }, + { + "Command": "CLUSTER_SYNC", + "Name": "CLUSTER|SYNC", + "IsInternal": true, + "Arity": 4, + "Flags": "Admin, NoMulti, NoScript", + "AclCategories": "Admin, Dangerous, Slow, Garnet" } ] }, diff --git a/libs/server/Resp/CmdStrings.cs b/libs/server/Resp/CmdStrings.cs index 31b4998718..30dadcdf5e 100644 --- a/libs/server/Resp/CmdStrings.cs +++ b/libs/server/Resp/CmdStrings.cs @@ -372,6 +372,7 @@ static partial class CmdStrings public static ReadOnlySpan mtasks => "MTASKS"u8; public static ReadOnlySpan aofsync => "AOFSYNC"u8; public static ReadOnlySpan appendlog => "APPENDLOG"u8; + public static ReadOnlySpan attach_sync => "ATTACH_SYNC"u8; public static ReadOnlySpan banlist => "BANLIST"u8; public static ReadOnlySpan begin_replica_recover => "BEGIN_REPLICA_RECOVER"u8; public static ReadOnlySpan endpoint => "ENDPOINT"u8; @@ -380,6 +381,7 @@ static partial class CmdStrings public static ReadOnlySpan initiate_replica_sync => "INITIATE_REPLICA_SYNC"u8; public static ReadOnlySpan send_ckpt_file_segment => "SEND_CKPT_FILE_SEGMENT"u8; public static ReadOnlySpan send_ckpt_metadata => "SEND_CKPT_METADATA"u8; + public static ReadOnlySpan cluster_sync => "SYNC"u8; // Lua scripting strings public static ReadOnlySpan LUA_OK => "OK"u8; diff --git a/libs/server/Resp/Parser/RespCommand.cs b/libs/server/Resp/Parser/RespCommand.cs index 09e312518f..60a7e9e3c8 100644 --- a/libs/server/Resp/Parser/RespCommand.cs +++ b/libs/server/Resp/Parser/RespCommand.cs @@ -317,6 +317,7 @@ public enum RespCommand : ushort CLUSTER_ADDSLOTSRANGE, CLUSTER_AOFSYNC, CLUSTER_APPENDLOG, + CLUSTER_ATTACH_SYNC, CLUSTER_BANLIST, CLUSTER_BEGIN_REPLICA_RECOVER, CLUSTER_BUMPEPOCH, @@ -354,7 +355,8 @@ public enum RespCommand : ushort CLUSTER_SETSLOTSRANGE, CLUSTER_SHARDS, CLUSTER_SLOTS, - CLUSTER_SLOTSTATE, // Note: Update IsClusterSubCommand if adding new cluster subcommands after this + CLUSTER_SLOTSTATE, + CLUSTER_SYNC, // Note: Update IsClusterSubCommand if adding new cluster subcommands after this // Don't require AUTH (if auth is enabled) AUTH, // Note: Update IsNoAuth if adding new no-auth commands before this @@ -591,7 +593,7 @@ public static bool IsClusterSubCommand(this RespCommand cmd) { // If cmd < RespCommand.CLUSTER_ADDSLOTS - underflows, setting high bits uint test = (uint)((int)cmd - (int)RespCommand.CLUSTER_ADDSLOTS); - bool inRange = test <= (RespCommand.CLUSTER_SLOTSTATE - RespCommand.CLUSTER_ADDSLOTS); + bool inRange = test <= (RespCommand.CLUSTER_SYNC - RespCommand.CLUSTER_ADDSLOTS); return inRange; } } @@ -2070,6 +2072,10 @@ private RespCommand SlowParseCommand(ref int count, ref ReadOnlySpan speci { return RespCommand.CLUSTER_APPENDLOG; } + else if (subCommand.SequenceEqual(CmdStrings.attach_sync)) + { + return RespCommand.CLUSTER_ATTACH_SYNC; + } else if (subCommand.SequenceEqual(CmdStrings.banlist)) { return RespCommand.CLUSTER_BANLIST; @@ -2106,6 +2112,10 @@ private RespCommand SlowParseCommand(ref int count, ref ReadOnlySpan speci { return RespCommand.CLUSTER_SEND_CKPT_METADATA; } + else if (subCommand.SequenceEqual(CmdStrings.cluster_sync)) + { + return RespCommand.CLUSTER_SYNC; + } string errMsg = string.Format(CmdStrings.GenericErrUnknownSubCommand, Encoding.UTF8.GetString(subCommand), diff --git a/playground/CommandInfoUpdater/SupportedCommand.cs b/playground/CommandInfoUpdater/SupportedCommand.cs index 3530c1b5a4..89fa785e50 100644 --- a/playground/CommandInfoUpdater/SupportedCommand.cs +++ b/playground/CommandInfoUpdater/SupportedCommand.cs @@ -58,6 +58,7 @@ public class SupportedCommand new("CLUSTER|ADDSLOTSRANGE", RespCommand.CLUSTER_ADDSLOTSRANGE), new("CLUSTER|AOFSYNC", RespCommand.CLUSTER_AOFSYNC), new("CLUSTER|APPENDLOG", RespCommand.CLUSTER_APPENDLOG), + new("CLUSTER|ATTACH_SYNC", RespCommand.CLUSTER_ATTACH_SYNC), new("CLUSTER|BANLIST", RespCommand.CLUSTER_BANLIST), new("CLUSTER|BEGIN_REPLICA_RECOVER", RespCommand.CLUSTER_BEGIN_REPLICA_RECOVER), new("CLUSTER|BUMPEPOCH", RespCommand.CLUSTER_BUMPEPOCH), @@ -95,7 +96,8 @@ public class SupportedCommand new("CLUSTER|SETSLOTSRANGE", RespCommand.CLUSTER_SETSLOTSRANGE), new("CLUSTER|SHARDS", RespCommand.CLUSTER_SHARDS), new("CLUSTER|SLOTS", RespCommand.CLUSTER_SLOTS), - new("CLUSTER|SLOTSTATE", RespCommand.CLUSTER_SLOTSTATE) + new("CLUSTER|SLOTSTATE", RespCommand.CLUSTER_SLOTSTATE), + new("CLUSTER|SYNC", RespCommand.CLUSTER_SYNC), ]), new("COMMAND", RespCommand.COMMAND, [ From ea1dfddd4644c65ce882f2b231b5cfde634da5fb Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Fri, 24 Jan 2025 16:55:26 -0800 Subject: [PATCH 06/32] adding test for diskless replication --- .../ClusterManagementTests.cs | 6 +- .../Garnet.test.cluster/ClusterTestContext.cs | 57 +++++++------ .../ClusterReplicationBaseTests.cs | 16 ++-- .../ClusterReplicationDiskless.cs | 79 +++++++++++++++++++ test/Garnet.test/Resp/ACL/RespCommandTests.cs | 77 ++++++++++++++++++ test/Garnet.test/TestUtils.cs | 15 ++-- 6 files changed, 211 insertions(+), 39 deletions(-) create mode 100644 test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs diff --git a/test/Garnet.test.cluster/ClusterManagementTests.cs b/test/Garnet.test.cluster/ClusterManagementTests.cs index 51ca6f950e..f60aa1a5ed 100644 --- a/test/Garnet.test.cluster/ClusterManagementTests.cs +++ b/test/Garnet.test.cluster/ClusterManagementTests.cs @@ -413,7 +413,7 @@ public void ClusterRestartNodeDropGossip() public void ClusterClientList() { const int NodeCount = 4; - context.CreateInstances(NodeCount, enableAOF: true, mainMemoryReplication: true, commitFrequencyMs: -1); + context.CreateInstances(NodeCount, enableAOF: true, MainMemoryReplication: true, CommitFrequencyMs: -1); context.CreateConnection(); _ = context.clusterTestUtils.SimpleSetupCluster(NodeCount / 2, 1, logger: context.logger); @@ -544,7 +544,7 @@ public void ClusterClientList() public void ClusterClientKill() { const int NodeCount = 4; - context.CreateInstances(NodeCount, enableAOF: true, mainMemoryReplication: true, commitFrequencyMs: -1); + context.CreateInstances(NodeCount, enableAOF: true, MainMemoryReplication: true, CommitFrequencyMs: -1); context.CreateConnection(); _ = context.clusterTestUtils.SimpleSetupCluster(NodeCount / 2, 1, logger: context.logger); @@ -561,7 +561,7 @@ public void ClusterClientKillSlave() // Test SLAVE separately - it's equivalent to REPLICA, but needed for compatibility const int NodeCount = 4; - context.CreateInstances(NodeCount, enableAOF: true, mainMemoryReplication: true, commitFrequencyMs: -1); + context.CreateInstances(NodeCount, enableAOF: true, MainMemoryReplication: true, CommitFrequencyMs: -1); context.CreateConnection(); _ = context.clusterTestUtils.SimpleSetupCluster(NodeCount / 2, 1, logger: context.logger); diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 011b0a25ed..8d25e0b43b 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -89,18 +89,26 @@ public void RegisterCustomTxn(string name, Func proc /// /// /// - /// - /// - /// - /// - /// - /// - /// - /// - /// + /// + /// + /// + /// + /// + /// + /// /// /// + /// /// + /// + /// + /// + /// + /// + /// + /// + /// + /// public void CreateInstances( int shards, bool cleanClusterConfig = true, @@ -111,13 +119,13 @@ public void CreateInstances( string pageSize = default, string segmentSize = "1g", bool enableAOF = false, - bool mainMemoryReplication = false, - bool onDemandCheckpoint = false, - string aofMemorySize = "64m", - int commitFrequencyMs = 0, - bool disableStorageTier = false, - bool enableIncrementalSnapshots = false, - bool fastCommit = true, + bool MainMemoryReplication = false, + bool OnDemandCheckpoint = false, + string AofMemorySize = "64m", + int CommitFrequencyMs = 0, + bool DisableStorageTier = false, + bool EnableIncrementalSnapshots = false, + bool FastCommit = true, int timeout = -1, bool useTLS = false, bool useAcl = false, @@ -128,6 +136,7 @@ public void CreateInstances( int metricsSamplingFrequency = 0, bool enableLua = false, bool asyncReplay = false, + bool enableDisklessSync = false, LuaMemoryManagementMode luaMemoryMode = LuaMemoryManagementMode.Native, string luaMemoryLimit = "") { @@ -149,13 +158,13 @@ public void CreateInstances( MemorySize: memorySize, PageSize: pageSize, SegmentSize: segmentSize, - MainMemoryReplication: mainMemoryReplication, - AofMemorySize: aofMemorySize, - CommitFrequencyMs: commitFrequencyMs, - DisableStorageTier: disableStorageTier, - OnDemandCheckpoint: onDemandCheckpoint, - EnableIncrementalSnapshots: enableIncrementalSnapshots, - FastCommit: fastCommit, + MainMemoryReplication: MainMemoryReplication, + AofMemorySize: AofMemorySize, + CommitFrequencyMs: CommitFrequencyMs, + DisableStorageTier: DisableStorageTier, + OnDemandCheckpoint: OnDemandCheckpoint, + EnableIncrementalSnapshots: EnableIncrementalSnapshots, + FastCommit: FastCommit, useAcl: useAcl, aclFile: credManager.aclFilePath, authUsername: clusterCreds.user, @@ -165,6 +174,7 @@ public void CreateInstances( metricsSamplingFrequency: metricsSamplingFrequency, enableLua: enableLua, asyncReplay: asyncReplay, + enableDisklessSync: enableDisklessSync, luaMemoryMode: luaMemoryMode, luaMemoryLimit: luaMemoryLimit); @@ -195,6 +205,7 @@ public void CreateInstances( /// /// /// + /// /// /// /// diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs index 4f7749a8a9..3c92592a50 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs @@ -314,7 +314,7 @@ void ClusterSRPrimaryCheckpointRetrieve(bool performRMW, bool disableObjects, bo var primary_count = 1; var nodes_count = primary_count + primary_count * replica_count; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, lowMemory: lowMemory, segmentSize: manySegments ? "4k" : "1g", disableStorageTier: disableStorageTier, enableIncrementalSnapshots: incrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: disableObjects, lowMemory: lowMemory, segmentSize: manySegments ? "4k" : "1g", DisableStorageTier: disableStorageTier, EnableIncrementalSnapshots: incrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -650,7 +650,7 @@ public void ClusterFailoverAttachReplicas([Values] bool performRMW, [Values] boo var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, enableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: true, EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var (shards, _) = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -731,7 +731,7 @@ public void ClusterReplicationCheckpointCleanupTest([Values] bool performRMW, [V var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, tryRecover: true, disableObjects: disableObjects, lowMemory: true, segmentSize: "4k", enableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, tryRecover: true, disableObjects: disableObjects, lowMemory: true, segmentSize: "4k", EnableIncrementalSnapshots: enableIncrementalSnapshots, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, [(0, 16383)], true, context.logger)); context.clusterTestUtils.BumpEpoch(0, logger: context.logger); @@ -764,7 +764,7 @@ public void ClusterMainMemoryReplicationAttachReplicas() var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, mainMemoryReplication: true, onDemandCheckpoint: true, commitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: true, MainMemoryReplication: true, OnDemandCheckpoint: true, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); ClassicAssert.AreEqual("OK", context.clusterTestUtils.AddDelSlotsRange(0, new List<(int, int)>() { (0, 16383) }, true)); @@ -808,7 +808,7 @@ public void ClusterDontKnowReplicaFailTest([Values] bool performRMW, [Values] bo var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: true, mainMemoryReplication: MainMemoryReplication, onDemandCheckpoint: onDemandCheckpoint, commitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: true, MainMemoryReplication: MainMemoryReplication, OnDemandCheckpoint: onDemandCheckpoint, CommitFrequencyMs: -1, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); var primaryNodeIndex = 0; @@ -909,7 +909,7 @@ void ClusterDivergentReplicasTest(bool performRMW, bool disableObjects, bool ckp var primary_count = 1; var nodes_count = primary_count + (primary_count * replica_count); ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: disableObjects, mainMemoryReplication: mainMemoryReplication, commitFrequencyMs: mainMemoryReplication ? -1 : 0, onDemandCheckpoint: mainMemoryReplication, fastCommit: fastCommit, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: disableObjects, MainMemoryReplication: mainMemoryReplication, CommitFrequencyMs: mainMemoryReplication ? -1 : 0, OnDemandCheckpoint: mainMemoryReplication, FastCommit: fastCommit, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); @@ -1043,7 +1043,7 @@ public void ClusterReplicateFails() ServerCredential userCreds = new(UserName, Password, IsAdmin: true, UsedForClusterAuth: false, IsClearText: true); context.GenerateCredentials([userCreds, clusterCreds]); - context.CreateInstances(2, disableObjects: true, disablePubSub: true, enableAOF: true, clusterCreds: clusterCreds, useAcl: true, mainMemoryReplication: true, commitFrequencyMs: -1, asyncReplay: asyncReplay); + context.CreateInstances(2, disableObjects: true, disablePubSub: true, enableAOF: true, clusterCreds: clusterCreds, useAcl: true, MainMemoryReplication: true, CommitFrequencyMs: -1, asyncReplay: asyncReplay); var primaryEndpoint = (IPEndPoint)context.endpoints.First(); var replicaEndpoint = (IPEndPoint)context.endpoints.Last(); @@ -1072,7 +1072,7 @@ public void ClusterReplicationCheckpointAlignmentTest([Values] bool performRMW) var primaryNodeIndex = 0; var replicaNodeIndex = 1; ClassicAssert.IsTrue(primary_count > 0); - context.CreateInstances(nodes_count, disableObjects: false, mainMemoryReplication: true, commitFrequencyMs: -1, onDemandCheckpoint: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); + context.CreateInstances(nodes_count, disableObjects: false, MainMemoryReplication: true, CommitFrequencyMs: -1, OnDemandCheckpoint: true, enableAOF: true, useTLS: useTLS, asyncReplay: asyncReplay); context.CreateConnection(useTLS: useTLS); _ = context.clusterTestUtils.SimpleSetupCluster(primary_count, replica_count, logger: context.logger); diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs new file mode 100644 index 0000000000..3eafb6ffff --- /dev/null +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs @@ -0,0 +1,79 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Collections.Generic; +using Microsoft.Extensions.Logging; +using NUnit.Framework; + +namespace Garnet.test.cluster +{ + /// + /// TODO: Testing scenarios + /// 1. Empty replica attach sync + /// a. Primary empty + /// b. Primary non-empty + /// 2. Replica same history and version different AOF + /// 3. Replica same history and different version and AOF + /// 4. Replica different history, version and AOF + /// + public class ClusterReplicationDiskless + { + ClusterTestContext context; + readonly int keyCount = 256; + + protected bool useTLS = false; + protected bool asyncReplay = false; + + public Dictionary monitorTests = new() + { + {"ClusterReplicationSimpleFailover", LogLevel.Warning}, + }; + + [SetUp] + public virtual void Setup() + { + context = new ClusterTestContext(); + context.Setup(monitorTests); + } + + [TearDown] + public virtual void TearDown() + { + context?.TearDown(); + } + + [Test, Order(1)] + [Category("REPLICATION")] + public void ClusterSimpleAttachSync([Values] bool disableObjects) + { + var nodes_count = 2; + var primaryIndex = 0; + var replicaIndex = 1; + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateConnection(useTLS: useTLS); + + // Setup primary and introduce it to future replica + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaIndex, replicaIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaIndex, logger: context.logger); + + // Populate Primary + var keyLength = 16; + var kvpairCount = keyCount; + context.kvPairs = []; + context.PopulatePrimary(ref context.kvPairs, keyLength, kvpairCount, 0); + + // Ensure node is known + context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); + + // Attach sync session + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + while (true) + { + + } + } + } +} \ No newline at end of file diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index 1b54c6c6a0..5b59c2bca8 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -4,6 +4,7 @@ using System; using System.Collections.Generic; using System.Globalization; +using System.IO; using System.Linq; using System.Numerics; using System.Reflection; @@ -1005,6 +1006,53 @@ static async Task DoClusterAppendLogAsync(GarnetClient client) } } + public async Task ClusterAttachSyncACLsAsync() + { + // All cluster command "success" is a thrown exception, because clustering is disabled + + await CheckCommandsAsync( + "CLUSTER ATTACH_SYNC", + [DoClusterAttachSyncAsync] + ); + + static async Task DoClusterAttachSyncAsync(GarnetClient client) + { + var ms = new MemoryStream(); + var writer = new BinaryWriter(ms, Encoding.ASCII); + // See SyncMetadata + writer.Write(0); + writer.Write(0); + + writer.Write(0); + writer.Write(0); + + writer.Write(0); + writer.Write(0); + + writer.Write(0); + + byte[] byteBuffer = ms.ToArray(); + writer.Dispose(); + ms.Dispose(); + + try + { + await client.ExecuteForStringResultAsync("CLUSTER", ["ATTACH_SYNC", Encoding.UTF8.GetString(byteBuffer)]); + Assert.Fail("Shouldn't be reachable, cluster isn't enabled"); + } + catch (Exception e) + { + if (e.Message == "ERR This instance has cluster support disabled") + { + return; + } + + throw; + } + } + } + + [Test] public async Task ClusterBanListACLsAsync() { @@ -1704,6 +1752,35 @@ static async Task DoClusterMigrateAsync(GarnetClient client) } } + [Test] + public async Task ClusterSyncACLsAsync() + { + // All cluster command "success" is a thrown exception, because clustering is disabled + + await CheckCommandsAsync( + "CLUSTER SYNC", + [DoClusterMigrateAsync] + ); + + static async Task DoClusterMigrateAsync(GarnetClient client) + { + try + { + await client.ExecuteForStringResultAsync("CLUSTER", ["SYNC", "a", "b", "c"]); + Assert.Fail("Shouldn't be reachable, cluster isn't enabled"); + } + catch (Exception e) + { + if (e.Message == "ERR This instance has cluster support disabled") + { + return; + } + + throw; + } + } + } + [Test] public async Task ClusterMTasksACLsAsync() { diff --git a/test/Garnet.test/TestUtils.cs b/test/Garnet.test/TestUtils.cs index 5725aafc68..59d9343a40 100644 --- a/test/Garnet.test/TestUtils.cs +++ b/test/Garnet.test/TestUtils.cs @@ -402,15 +402,16 @@ public static GarnetServer[] CreateGarnetCluster( int metricsSamplingFrequency = 0, bool enableLua = false, bool asyncReplay = false, + bool enableDisklessSync = false, LuaMemoryManagementMode luaMemoryMode = LuaMemoryManagementMode.Native, string luaMemoryLimit = "") { if (UseAzureStorage) IgnoreIfNotRunningAzureTests(); - GarnetServer[] nodes = new GarnetServer[endpoints.Count]; - for (int i = 0; i < nodes.Length; i++) + var nodes = new GarnetServer[endpoints.Count]; + for (var i = 0; i < nodes.Length; i++) { - IPEndPoint endpoint = (IPEndPoint)endpoints[i]; + var endpoint = (IPEndPoint)endpoints[i]; var opts = GetGarnetServerOptions( checkpointDir, @@ -446,6 +447,7 @@ public static GarnetServer[] CreateGarnetCluster( metricsSamplingFrequency: metricsSamplingFrequency, enableLua: enableLua, asyncReplay: asyncReplay, + enableDisklessSync: enableDisklessSync, luaMemoryMode: luaMemoryMode, luaMemoryLimit: luaMemoryLimit); @@ -453,7 +455,7 @@ public static GarnetServer[] CreateGarnetCluster( if (opts.EndPoint is IPEndPoint ipEndpoint) { - int iter = 0; + var iter = 0; while (!IsPortAvailable(ipEndpoint.Port)) { ClassicAssert.Less(30, iter, "Failed to connect within 30 seconds"); @@ -500,6 +502,7 @@ public static GarnetServerOptions GetGarnetServerOptions( int metricsSamplingFrequency = 0, bool enableLua = false, bool asyncReplay = false, + bool enableDisklessSync = false, ILogger logger = null, LuaMemoryManagementMode luaMemoryMode = LuaMemoryManagementMode.Native, string luaMemoryLimit = "", @@ -606,7 +609,9 @@ public static GarnetServerOptions GetGarnetServerOptions( EnableLua = enableLua, ReplicationOffsetMaxLag = asyncReplay ? -1 : 0, LuaOptions = enableLua ? new LuaOptions(luaMemoryMode, luaMemoryLimit) : null, - UnixSocketPath = unixSocketPath + UnixSocketPath = unixSocketPath, + ReplicaDisklessSync = enableDisklessSync, + ReplicaDisklessSyncDelay = 1, }; if (lowMemory) From dbfa67ba1abd1d28622534ad38a44956203cfda2 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 12:34:15 -0800 Subject: [PATCH 07/32] update gcs extension to clearly mark logging progress --- .../ClientSession/GarnetClientSessionIncremental.cs | 13 ++++++++++--- .../GarnetClientSessionMigrationExtensions.cs | 1 + .../GarnetClientSessionReplicationExtensions.cs | 1 + 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/libs/client/ClientSession/GarnetClientSessionIncremental.cs b/libs/client/ClientSession/GarnetClientSessionIncremental.cs index 35b91ad1fe..d12a4e1221 100644 --- a/libs/client/ClientSession/GarnetClientSessionIncremental.cs +++ b/libs/client/ClientSession/GarnetClientSessionIncremental.cs @@ -12,8 +12,15 @@ namespace Garnet.client { + enum IncrementalSendType : byte + { + MIGRATE, + SYNC + } + public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageConsumer { + IncrementalSendType ist; bool isMainStore; byte* curr, head; int keyValuePairCount; @@ -183,9 +190,9 @@ private void TrackIterationProgress(int keyCount, int size, bool completed = fal var duration = TimeSpan.FromTicks(Stopwatch.GetTimestamp() - lastLog); if (completed || lastLog == 0 || duration >= iterationProgressFreq) { - logger?.LogTrace("[{op}]: isMainStore:({storeType}) totalKeyCount:({totalKeyCount}), totalPayloadSize:({totalPayloadSize} KB)", - completed ? "COMPLETED" : "MIGRATING", - isMainStore, + logger?.LogTrace("[{op}]: store:({storeType}) totalKeyCount:({totalKeyCount}), totalPayloadSize:({totalPayloadSize} KB)", + completed ? "COMPLETED" : ist, + isMainStore ? "MAIN STORE" : "OBJECT STORE", totalKeyCount.ToString("N0"), ((long)((double)totalPayloadSize / 1024)).ToString("N0")); lastLog = Stopwatch.GetTimestamp(); diff --git a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs index b23f56d34a..7662b533f8 100644 --- a/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionMigrationExtensions.cs @@ -176,6 +176,7 @@ public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMa tcsQueue.Enqueue(currTcsIterationTask); curr = offset; this.isMainStore = isMainStore; + this.ist = IncrementalSendType.MIGRATE; var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE; var replaceOption = replace ? T : F; diff --git a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs index 25115d7e76..408471734b 100644 --- a/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs +++ b/libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs @@ -414,6 +414,7 @@ public void SetClusterSyncHeader(string sourceNodeId, bool isMainStore) tcsQueue.Enqueue(currTcsIterationTask); curr = offset; this.isMainStore = isMainStore; + this.ist = IncrementalSendType.SYNC; var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE; var arraySize = 5; From 112a197cd7d979bc1d6f959142cdfa11eab6c6d3 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 12:39:58 -0800 Subject: [PATCH 08/32] fix gcs dispose on diskless attach, call dispose of replicationSyncManager, add more logging --- .../DisklessReplication/ReplicaSyncSession.cs | 13 +++- .../ReplicationSnapshotIterator.cs | 18 +++-- .../ReplicationSyncManager.cs | 77 ++++++++++--------- .../PrimaryOps/ReplicaSyncSession.cs | 1 + .../ReplicaOps/ReplicaDisklessSync.cs | 4 +- .../Server/Replication/ReplicationManager.cs | 2 + .../Session/RespClusterReplicationCommands.cs | 1 + libs/server/StoreWrapper.cs | 2 +- 8 files changed, 74 insertions(+), 44 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index 74a1fca75c..504a36bd3e 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -69,11 +69,20 @@ public Task ExecuteAsync(params string[] commands) return aofSyncTask.garnetClient.ExecuteAsync(commands); } + /// + /// Initialize iteration buffer + /// + public void InitializeIterationBuffer() + { + if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + aofSyncTask.garnetClient.InitializeIterationBuffer(); + } + /// /// Set Cluster Sync header /// /// - public void SetClysterSyncHeader(bool isMainStore) + public void SetClusterSyncHeader(bool isMainStore) { if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); if (aofSyncTask.garnetClient.NeedsInitialization) @@ -222,6 +231,8 @@ public async Task WaitForFlush() /// public async Task BeginAofSync() { + var aofSyncTask = this.aofSyncTask; + this.aofSyncTask = null; var mmr = clusterProvider.serverOptions.MainMemoryReplication; var aofNull = clusterProvider.serverOptions.UseAofNullDevice; diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index 49d6a36309..b37f13101f 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -34,7 +34,7 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ti numSessions = sessionStore.GetNumSessions(); mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); - if (!replicationSyncManager.GetClusterProvider.serverOptions.DisableObjects) + if (!replicationSyncManager.ClusterProvider.serverOptions.DisableObjects) objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this); } @@ -62,13 +62,17 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio for (var i = 0; i < numSessions; i++) { if (!IsActive(sessions, i)) continue; - sessions[i].SetClysterSyncHeader(isMainStore: isMainStore); + sessions[i].InitializeIterationBuffer(); + sessions[i].SetClusterSyncHeader(isMainStore: isMainStore); if (isMainStore) sessions[i].currentStoreVersion = targetVersion; else sessions[i].currentObjectStoreVersion = targetVersion; } + logger?.LogTrace("{OnStart} {store} {token} {currentVersion} {targetVersion}", + nameof(OnStart), isMainStore ? "MAIN STORE" : "OBJECT STORE", checkpointToken, currentVersion, targetVersion); + return true; } @@ -96,7 +100,7 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe if (!IsActive(sessions, i)) continue; // Initialize header if necessary - sessions[i].SetClysterSyncHeader(isMainStore: true); + sessions[i].SetClusterSyncHeader(isMainStore: true); // Try to write to network buffer. If failed we need to retry if (!sessions[i].TryWriteKeyValueSpanByte(ref key, ref value, out var task)) @@ -110,6 +114,7 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe // Wait for flush to complete for all and retry to enqueue previous keyValuePair above WaitForFlushAll(); + needToFlush = false; } return true; @@ -130,7 +135,7 @@ public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recor if (!IsActive(sessions, i)) continue; // Initialize header if necessary - sessions[i].SetClysterSyncHeader(isMainStore: false); + sessions[i].SetClusterSyncHeader(isMainStore: false); // Try to write to network buffer. If failed we need to retry if (!sessions[i].TryWriteKeyValueByteArray(key, objectData, value.Expiration, out var task)) @@ -162,7 +167,10 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long WaitForFlushAll(); // Enqueue version change commit - replicationSyncManager.GetClusterProvider.storeWrapper.EnqueueCommit(isMainStore, targetVersion); + replicationSyncManager.ClusterProvider.storeWrapper.EnqueueCommit(isMainStore, targetVersion); + + logger?.LogTrace("{OnStop} {store} {numberOfRecords} {targetVersion}", + nameof(OnStop), isMainStore ? "MAIN STORE" : "OBJECT STORE", numberOfRecords, targetVersion); } } diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index b6383e3a30..a19254704d 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -19,12 +19,12 @@ internal sealed class ReplicationSyncManager public ReplicaSyncSessionTaskStore GetSessionStore { get; } - public ClusterProvider GetClusterProvider { get; } + public ClusterProvider ClusterProvider { get; } public ReplicationSyncManager(ClusterProvider clusterProvider, ILogger logger = null) { GetSessionStore = new ReplicaSyncSessionTaskStore(clusterProvider.storeWrapper, clusterProvider, logger); - this.GetClusterProvider = clusterProvider; + ClusterProvider = clusterProvider; this.logger = logger; var opts = clusterProvider.serverOptions; @@ -47,7 +47,7 @@ public void Dispose() /// public bool AddSyncSession(SyncMetadata replicaSyncMetadata, out ReplicaSyncSession replicaSyncSession) { - replicaSyncSession = new ReplicaSyncSession(GetClusterProvider.storeWrapper, GetClusterProvider, replicaSyncMetadata, logger: logger); + replicaSyncSession = new ReplicaSyncSession(ClusterProvider.storeWrapper, ClusterProvider, replicaSyncMetadata, logger: logger); replicaSyncSession.SetStatus(SyncStatus.INITIALIZING); try { @@ -67,42 +67,49 @@ public bool AddSyncSession(SyncMetadata replicaSyncMetadata, out ReplicaSyncSess /// public async Task MainDisklessSync(ReplicaSyncSession replicaSyncSession) { - // Give opportunity to other replicas to attach for streaming sync - if (GetClusterProvider.serverOptions.ReplicaDisklessSyncDelay > 0) - Thread.Sleep(TimeSpan.FromSeconds(GetClusterProvider.serverOptions.ReplicaDisklessSyncDelay)); + try + { + // Give opportunity to other replicas to attach for streaming sync + if (ClusterProvider.serverOptions.ReplicaDisklessSyncDelay > 0) + Thread.Sleep(TimeSpan.FromSeconds(ClusterProvider.serverOptions.ReplicaDisklessSyncDelay)); - // Started syncing - replicaSyncSession.SetStatus(SyncStatus.INPROGRESS); + // Started syncing + replicaSyncSession.SetStatus(SyncStatus.INPROGRESS); - // Only one thread will acquire this lock - if (syncInProgress.OneWriteLock()) - { - // Launch a background task to sync the attached replicas using streaming snapshot - _ = Task.Run(() => StreamingSnapshotSync()); - } + // Only one thread will acquire this lock + if (syncInProgress.OneWriteLock()) + { + // Launch a background task to sync the attached replicas using streaming snapshot + _ = Task.Run(() => StreamingSnapshotSync()); + } + + // Wait for main sync task to complete + await replicaSyncSession.CompletePending(); + + // If session faulted return early + if (replicaSyncSession.Failed) + { + replicaSyncSession.LogError(); + replicaSyncSession.Dispose(); + return replicaSyncSession.GetSyncStatusInfo; + } - // Wait for main sync task to complete - await replicaSyncSession.CompletePending(); + await replicaSyncSession.BeginAofSync(); - // If session faulted return early - if (replicaSyncSession.Failed) + return replicaSyncSession.GetSyncStatusInfo; + } + finally { - replicaSyncSession.LogError(); replicaSyncSession.Dispose(); - return replicaSyncSession.GetSyncStatusInfo; } - - await replicaSyncSession.BeginAofSync(); - - return replicaSyncSession.GetSyncStatusInfo; } // Main streaming snapshot task async Task StreamingSnapshotSync() { // Parameters for sync operation - var disklessRepl = GetClusterProvider.serverOptions.ReplicaDisklessSync; - var disableObjects = GetClusterProvider.serverOptions.DisableObjects; + var disklessRepl = ClusterProvider.serverOptions.ReplicaDisklessSync; + var disableObjects = ClusterProvider.serverOptions.DisableObjects; // Replica sync session var numSessions = GetSessionStore.GetNumSessions(); @@ -111,7 +118,7 @@ async Task StreamingSnapshotSync() try { // Take lock to ensure no other task will be taking a checkpoint - while (!GetClusterProvider.storeWrapper.TryPauseCheckpoints()) + while (!ClusterProvider.storeWrapper.TryPauseCheckpoints()) await Task.Yield(); // Get sync metadata for checkpoint @@ -130,7 +137,7 @@ async Task StreamingSnapshotSync() GetSessionStore.Clear(); // Release checkpoint lock - GetClusterProvider.storeWrapper.ResumeCheckpoints(); + ClusterProvider.storeWrapper.ResumeCheckpoints(); // Unlock sync session lock syncInProgress.WriteUnlock(); @@ -145,10 +152,10 @@ async Task PrepareForSync() while (true) { // Calculate minimum address from which replicas should start streaming from - var syncFromAddress = GetClusterProvider.storeWrapper.appendOnlyFile.TailAddress; + var syncFromAddress = ClusterProvider.storeWrapper.appendOnlyFile.TailAddress; // Lock AOF address for sync streaming - if (GetClusterProvider.replicationManager.TryAddReplicationTasks(GetSessionStore.GetSessions(), syncFromAddress)) + if (ClusterProvider.replicationManager.TryAddReplicationTasks(GetSessionStore.GetSessions(), syncFromAddress)) break; // Retry if failed to lock AOF address because truncation occurred @@ -165,8 +172,8 @@ async Task PrepareForSync() sessions[i].Connect(); // Set store version to operate on - sessions[i].currentStoreVersion = GetClusterProvider.storeWrapper.store.CurrentVersion; - sessions[i].currentObjectStoreVersion = disableObjects ? -1 : GetClusterProvider.storeWrapper.objectStore.CurrentVersion; + 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 @@ -201,13 +208,13 @@ async Task StreamDisklessCheckpoint() var manager = new SnapshotIteratorManager(this, clusterTimeout, cts.Token, logger); // Iterate through main store - var mainStoreResult = await GetClusterProvider.storeWrapper.store. + var mainStoreResult = await ClusterProvider.storeWrapper.store. TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); - if (!GetClusterProvider.serverOptions.DisableObjects) + if (!ClusterProvider.serverOptions.DisableObjects) { // Iterate through object store - var objectStoreResult = await GetClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); } } } diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index 0e2ca027d8..4cb314fa98 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -46,6 +46,7 @@ internal sealed partial class ReplicaSyncSession( public void Dispose() { + aofSyncTask = null; cts.Cancel(); cts.Dispose(); semaphore?.Dispose(); diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 9895ca4622..51742a7293 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -37,7 +37,6 @@ public bool TryReplicateDisklessSync( // Wait for threads to agree configuration change of this node session.UnsafeBumpAndWaitForEpochTransition(); - _ = Task.Run(() => TryBeginReplicaSync()); } catch (Exception ex) @@ -50,6 +49,7 @@ async Task TryBeginReplicaSync() { var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync; var dissableObjects = clusterProvider.serverOptions.DisableObjects; + GarnetClientSession gcs = null; try { if (!clusterProvider.serverOptions.EnableFastCommit) @@ -74,7 +74,6 @@ async Task TryBeginReplicaSync() // Retrieval completion coordinated by remoteCheckpointRetrievalCompleted var current = clusterProvider.clusterManager.CurrentConfig; var (address, port) = current.GetLocalNodePrimaryAddress(); - GarnetClientSession gcs = null; CheckpointEntry checkpointEntry = null; if (!disklessSync) @@ -123,6 +122,7 @@ async Task TryBeginReplicaSync() } finally { + gcs?.Dispose(); recvCheckpointHandler?.Dispose(); } } diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index ac10722efa..e70dd3614d 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -198,6 +198,8 @@ public void Dispose() replicationConfigDevice?.Dispose(); replicationConfigDevicePool?.Free(); + replicationSyncManager?.Dispose(); + checkpointStore.WaitForReplicas(); replicaSyncSessionTaskStore.Dispose(); replicaReplayTaskCts.Cancel(); diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index 507e64f59b..286f85e352 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -460,6 +460,7 @@ private bool NetworkClusterSync(out bool invalidParameters) } else if (storeTypeSpan.EqualsUpperCaseSpanIgnoringCase("OSTORE"u8)) { + TrackImportProgress(keyValuePairCount, isMainStore: false, keyValuePairCount == 0); while (i < keyValuePairCount) { if (!RespReadUtils.TryReadSerializedData(out var key, out var data, out var expiration, ref payloadPtr, payloadEndPtr)) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index c4745e6753..fa99cfe8dd 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -700,7 +700,7 @@ private bool GrowIndexIfNeeded(StoreType storeType, long indexMaxSize, long over /// public void Dispose() { - //Wait for checkpoints to complete and disable checkpointing + // Wait for checkpoints to complete and disable checkpointing _checkpointTaskLock.WriteLock(); itemBroker?.Dispose(); From a20ac4694a5fe364296928ab79b1c28d5db92f69 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 12:41:03 -0800 Subject: [PATCH 09/32] complete first diskless replication test --- test/Garnet.test.cluster/ClusterTestContext.cs | 4 ++-- .../ReplicationTests/ClusterReplicationDiskless.cs | 7 ++++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 8d25e0b43b..5da3420a06 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -65,9 +65,9 @@ public void TearDown() waiter?.Dispose(); clusterTestUtils?.Dispose(); loggerFactory?.Dispose(); - if (!Task.Run(() => DisposeCluster()).Wait(TimeSpan.FromSeconds(15))) + if (!Task.Run(() => DisposeCluster()).Wait(TimeSpan.FromSeconds(60))) logger?.LogError("Timed out waiting for DisposeCluster"); - if (!Task.Run(() => TestUtils.DeleteDirectory(TestFolder, true)).Wait(TimeSpan.FromSeconds(15))) + if (!Task.Run(() => TestUtils.DeleteDirectory(TestFolder, true)).Wait(TimeSpan.FromSeconds(60))) logger?.LogError("Timed out waiting for DisposeCluster"); } diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs index 3eafb6ffff..5865acef7b 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs @@ -70,10 +70,11 @@ public void ClusterSimpleAttachSync([Values] bool disableObjects) // Attach sync session _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); - while (true) - { + // Wait for replica to catch up + context.clusterTestUtils.WaitForReplicaAofSync(primaryIndex, replicaIndex, logger: context.logger); - } + // Validate replica data + context.ValidateKVCollectionAgainstReplica(ref context.kvPairs, replicaIndex: replicaIndex, primaryIndex: primaryIndex); } } } \ No newline at end of file From 471c73c9c081a3cc0ddc48744c1122e271930b65 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 14:10:08 -0800 Subject: [PATCH 10/32] fix iterator check for null when empty store --- .../DisklessReplication/ReplicaSyncSession.cs | 33 ++++++++++--------- .../ReplicationSnapshotIterator.cs | 1 - 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index 504a36bd3e..eeb860e9dd 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -65,7 +65,7 @@ public void Connect() /// public Task ExecuteAsync(params string[] commands) { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); return aofSyncTask.garnetClient.ExecuteAsync(commands); } @@ -74,7 +74,7 @@ public Task ExecuteAsync(params string[] commands) /// public void InitializeIterationBuffer() { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); aofSyncTask.garnetClient.InitializeIterationBuffer(); } @@ -84,7 +84,7 @@ public void InitializeIterationBuffer() /// public void SetClusterSyncHeader(bool isMainStore) { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); if (aofSyncTask.garnetClient.NeedsInitialization) aofSyncTask.garnetClient.SetClusterSyncHeader(clusterProvider.clusterManager.CurrentConfig.LocalNodeId, isMainStore: isMainStore); } @@ -98,7 +98,7 @@ public void SetClusterSyncHeader(bool isMainStore) /// public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out Task task) { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); return aofSyncTask.garnetClient.TryWriteKeyValueSpanByte(ref key, ref value, out task); } @@ -112,7 +112,7 @@ public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out T /// public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, out Task task) { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); return aofSyncTask.garnetClient.TryWriteKeyValueByteArray(key, value, expiration, out task); } @@ -122,7 +122,7 @@ public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, /// public void SendAndResetIterationBuffer(TimeSpan timeout, CancellationToken token) { - if (flushTask != null) WaitForFlush().GetAwaiter().GetResult(); + WaitForFlush().GetAwaiter().GetResult(); SetFlushTask(aofSyncTask.garnetClient.SendAndResetIterationBuffer(), timeout: timeout, token: token); } #endregion @@ -196,16 +196,19 @@ public void SetStatus(SyncStatus status, string error = null) /// public void SetFlushTask(Task task, TimeSpan timeout, CancellationToken token) { - flushTask = task.ContinueWith(resp => + if (task != null) { - if (!resp.Result.Equals("OK", StringComparison.Ordinal)) + flushTask = task.ContinueWith(resp => { - logger?.LogError("ReplicaSyncSession: {errorMsg}", resp.Result); - SetStatus(SyncStatus.FAILED, resp.Result); - return false; - } - return true; - }, TaskContinuationOptions.OnlyOnRanToCompletion).WaitAsync(timeout, token); + if (!resp.Result.Equals("OK", StringComparison.Ordinal)) + { + logger?.LogError("ReplicaSyncSession: {errorMsg}", resp.Result); + SetStatus(SyncStatus.FAILED, resp.Result); + return false; + } + return true; + }, TaskContinuationOptions.OnlyOnRanToCompletion).WaitAsync(timeout, token); + } } /// @@ -216,7 +219,7 @@ public async Task WaitForFlush() { try { - _ = await flushTask; + if (flushTask != null) _ = await flushTask; flushTask = null; } catch (Exception ex) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index b37f13101f..b182e40dc3 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -63,7 +63,6 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio { if (!IsActive(sessions, i)) continue; sessions[i].InitializeIterationBuffer(); - sessions[i].SetClusterSyncHeader(isMainStore: isMainStore); if (isMainStore) sessions[i].currentStoreVersion = targetVersion; else From 9ecb023bf196133818dc432235e20ccee0780483 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 16:05:38 -0800 Subject: [PATCH 11/32] fix iterator for object store cluster sync --- libs/cluster/Session/RespClusterReplicationCommands.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/libs/cluster/Session/RespClusterReplicationCommands.cs b/libs/cluster/Session/RespClusterReplicationCommands.cs index 286f85e352..fdb1aebf54 100644 --- a/libs/cluster/Session/RespClusterReplicationCommands.cs +++ b/libs/cluster/Session/RespClusterReplicationCommands.cs @@ -469,6 +469,7 @@ private bool NetworkClusterSync(out bool invalidParameters) var value = clusterProvider.storeWrapper.GarnetObjectSerializer.Deserialize(data); value.Expiration = expiration; _ = basicGarnetApi.SET(key, value); + i++; } } From 2e59fb2f9eb33326408d2894addf858a04d37605 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Mon, 27 Jan 2025 16:05:58 -0800 Subject: [PATCH 12/32] add simple diskless sync test --- .../Garnet.test.cluster/ClusterTestContext.cs | 10 ++--- .../ClusterReplicationBaseTests.cs | 2 +- .../ClusterReplicationDiskless.cs | 40 +++++++++++++++---- 3 files changed, 38 insertions(+), 14 deletions(-) diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 5da3420a06..40a8abd828 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -472,23 +472,23 @@ public void ValidateKVCollectionAgainstReplica( } } - public void ValidateNodeObjects(ref Dictionary> kvPairsObj, int nodeIndex, bool set = false) + public void ValidateNodeObjects(ref Dictionary> kvPairsObj, int replicaIndex, bool set = false) { foreach (var key in kvPairsObj.Keys) { var elements = kvPairsObj[key]; List result; if (!set) - result = clusterTestUtils.Lrange(nodeIndex, key, logger); + result = clusterTestUtils.Lrange(replicaIndex, key, logger); else - result = clusterTestUtils.Smembers(nodeIndex, key, logger); + result = clusterTestUtils.Smembers(replicaIndex, key, logger); while (result.Count == 0) { if (!set) - result = clusterTestUtils.Lrange(nodeIndex, key, logger); + result = clusterTestUtils.Lrange(replicaIndex, key, logger); else - result = clusterTestUtils.Smembers(nodeIndex, key, logger); + result = clusterTestUtils.Smembers(replicaIndex, key, logger); ClusterTestUtils.BackOff(cancellationToken: cts.Token); } if (!set) diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs index 3c92592a50..bb83c6bf26 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationBaseTests.cs @@ -1025,7 +1025,7 @@ void ClusterDivergentReplicasTest(bool performRMW, bool disableObjects, bool ckp if (disableObjects) context.ValidateKVCollectionAgainstReplica(ref context.kvPairs, replicaIndex: replicaIndex, primaryIndex: newPrimaryIndex); else - context.ValidateNodeObjects(ref context.kvPairsObj, nodeIndex: newPrimaryIndex, set: set); + context.ValidateNodeObjects(ref context.kvPairsObj, replicaIndex: newPrimaryIndex, set: set); } [Test, Order(21)] diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs index 5865acef7b..d2a29fcf84 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs @@ -42,9 +42,36 @@ public virtual void TearDown() context?.TearDown(); } + void PopulatePrimary(int primaryIndex, bool disableObjects, bool performRMW) + { + context.kvPairs = []; + context.kvPairsObj = []; + var addCount = 5; + var keyLength = 16; + var kvpairCount = keyCount; + context.kvPairs = []; + context.kvPairsObj = []; + // New insert + if (!performRMW) + context.PopulatePrimary(ref context.kvPairs, keyLength, kvpairCount, primaryIndex: primaryIndex); + else + context.PopulatePrimaryRMW(ref context.kvPairs, keyLength, kvpairCount, primaryIndex: 0, addCount); + + if (!disableObjects) + context.PopulatePrimaryWithObjects(ref context.kvPairsObj, keyLength, kvpairCount, primaryIndex: 0); + } + + void Validate(int primaryIndex, int replicaIndex, bool disableObjects) + { + // Validate replica data + context.ValidateKVCollectionAgainstReplica(ref context.kvPairs, replicaIndex: replicaIndex, primaryIndex: primaryIndex); + if (disableObjects) + context.ValidateNodeObjects(ref context.kvPairsObj, replicaIndex); + } + [Test, Order(1)] [Category("REPLICATION")] - public void ClusterSimpleAttachSync([Values] bool disableObjects) + public void ClusterSimpleAttachDisklessSync([Values] bool disableObjects, [Values] bool performRMW) { var nodes_count = 2; var primaryIndex = 0; @@ -58,15 +85,12 @@ public void ClusterSimpleAttachSync([Values] bool disableObjects) context.clusterTestUtils.SetConfigEpoch(replicaIndex, replicaIndex + 1, logger: context.logger); context.clusterTestUtils.Meet(primaryIndex, replicaIndex, logger: context.logger); - // Populate Primary - var keyLength = 16; - var kvpairCount = keyCount; - context.kvPairs = []; - context.PopulatePrimary(ref context.kvPairs, keyLength, kvpairCount, 0); - // Ensure node is known context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); + // Populate Primary + PopulatePrimary(primaryIndex, disableObjects, performRMW); + // Attach sync session _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); @@ -74,7 +98,7 @@ public void ClusterSimpleAttachSync([Values] bool disableObjects) context.clusterTestUtils.WaitForReplicaAofSync(primaryIndex, replicaIndex, logger: context.logger); // Validate replica data - context.ValidateKVCollectionAgainstReplica(ref context.kvPairs, replicaIndex: replicaIndex, primaryIndex: primaryIndex); + Validate(primaryIndex, replicaIndex, disableObjects); } } } \ No newline at end of file From a6e6036ea044ea852f40b8b6b51f0311fcf4e7aa Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 28 Jan 2025 10:50:57 -0800 Subject: [PATCH 13/32] cleanup code --- .../Replication/PrimaryOps/AofTaskStore.cs | 8 +- .../DisklessReplication/ReplicaSyncSession.cs | 200 +++++++++--------- .../ReplicationSyncManager.cs | 4 +- .../DisklessReplication/SyncStatus.cs | 6 + .../PrimaryOps/ReplicaSyncSession.cs | 2 +- 5 files changed, 115 insertions(+), 105 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs index fffbf21048..dbd66816e5 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs @@ -281,7 +281,7 @@ public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, lon Debug.Assert(t != null); if (t.remoteNodeId == rss.replicaNodeId) { - tasks[i] = rss.GetAofSyncTask; + tasks[i] = rss.AofSyncTask; t.Dispose(); added = true; break; @@ -301,7 +301,7 @@ public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, lon Array.Clear(old_tasks); } // Add new AOF sync task - tasks[numTasks++] = rss.GetAofSyncTask; + tasks[numTasks++] = rss.AofSyncTask; } success = true; @@ -315,9 +315,9 @@ public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, lon foreach (var rss in replicaSyncSessions) { if (rss == null) continue; - if (rss.GetAofSyncTask != null) + if (rss.AofSyncTask != null) { - rss.GetAofSyncTask.Dispose(); + rss.AofSyncTask.Dispose(); } } } diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index eeb860e9dd..d1fca08c72 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -14,14 +14,17 @@ internal sealed partial class ReplicaSyncSession { SyncStatusInfo ssInfo; Task flushTask; - AofSyncTaskInfo aofSyncTask = null; - bool sendMainStore = false; bool sendObjectStore = false; bool truncatedAof = false; bool fullSync = false; - public bool IsConnected => aofSyncTask != null && aofSyncTask.IsConnected; + /// + /// Get the associated aof sync task instance with this replica sync session + /// + public AofSyncTaskInfo AofSyncTask { get; private set; } = null; + + public bool IsConnected => AofSyncTask != null && AofSyncTask.IsConnected; public bool Failed => ssInfo.syncStatus == SyncStatus.FAILED; @@ -54,8 +57,8 @@ public void LogError() /// public void Connect() { - if (!aofSyncTask.IsConnected) - aofSyncTask.garnetClient.Connect(); + if (!AofSyncTask.IsConnected) + AofSyncTask.garnetClient.Connect(); } /// @@ -66,7 +69,7 @@ public void Connect() public Task ExecuteAsync(params string[] commands) { WaitForFlush().GetAwaiter().GetResult(); - return aofSyncTask.garnetClient.ExecuteAsync(commands); + return AofSyncTask.garnetClient.ExecuteAsync(commands); } /// @@ -75,7 +78,7 @@ public Task ExecuteAsync(params string[] commands) public void InitializeIterationBuffer() { WaitForFlush().GetAwaiter().GetResult(); - aofSyncTask.garnetClient.InitializeIterationBuffer(); + AofSyncTask.garnetClient.InitializeIterationBuffer(); } /// @@ -85,8 +88,8 @@ public void InitializeIterationBuffer() public void SetClusterSyncHeader(bool isMainStore) { WaitForFlush().GetAwaiter().GetResult(); - if (aofSyncTask.garnetClient.NeedsInitialization) - aofSyncTask.garnetClient.SetClusterSyncHeader(clusterProvider.clusterManager.CurrentConfig.LocalNodeId, isMainStore: isMainStore); + if (AofSyncTask.garnetClient.NeedsInitialization) + AofSyncTask.garnetClient.SetClusterSyncHeader(clusterProvider.clusterManager.CurrentConfig.LocalNodeId, isMainStore: isMainStore); } /// @@ -99,7 +102,7 @@ public void SetClusterSyncHeader(bool isMainStore) public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out Task task) { WaitForFlush().GetAwaiter().GetResult(); - return aofSyncTask.garnetClient.TryWriteKeyValueSpanByte(ref key, ref value, out task); + return AofSyncTask.garnetClient.TryWriteKeyValueSpanByte(ref key, ref value, out task); } /// @@ -113,7 +116,7 @@ public bool TryWriteKeyValueSpanByte(ref SpanByte key, ref SpanByte value, out T public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, out Task task) { WaitForFlush().GetAwaiter().GetResult(); - return aofSyncTask.garnetClient.TryWriteKeyValueByteArray(key, value, expiration, out task); + return AofSyncTask.garnetClient.TryWriteKeyValueByteArray(key, value, expiration, out task); } /// @@ -123,7 +126,7 @@ public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, public void SendAndResetIterationBuffer(TimeSpan timeout, CancellationToken token) { WaitForFlush().GetAwaiter().GetResult(); - SetFlushTask(aofSyncTask.garnetClient.SendAndResetIterationBuffer(), timeout: timeout, token: token); + SetFlushTask(AofSyncTask.garnetClient.SendAndResetIterationBuffer(), timeout: timeout, token: token); } #endregion @@ -131,50 +134,7 @@ public void SendAndResetIterationBuffer(TimeSpan timeout, CancellationToken toke /// Associated aof sync task instance with this replica sync session /// /// - public void AddAofSyncTask(AofSyncTaskInfo aofSyncTask) => this.aofSyncTask = aofSyncTask; - - /// - /// Get the associated aof sync task instance with this replica sync session - /// - public AofSyncTaskInfo GetAofSyncTask => aofSyncTask; - - /// - /// Wait until sync of checkpoint is completed - /// - /// - public async Task CompletePending() - { - while (ssInfo.syncStatus == SyncStatus.INPROGRESS) - await Task.Yield(); - } - - /// - /// Should stream - /// - /// - public bool ShouldStreamCheckpoint() - { - // TODO: implement disk-based logic if possible - return clusterProvider.serverOptions.ReplicaDisklessSync ? - ShouldStreamDisklessCheckpoint() : true; - - bool ShouldStreamDisklessCheckpoint() - { - var localPrimaryReplId = clusterProvider.replicationManager.PrimaryReplId; - var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal); - sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion; - sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion; - truncatedAof = replicaSyncMetadata.currentAofTailAddress < aofSyncTask.StartAddress; - - // 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 - fullSync = sendMainStore || sendObjectStore || truncatedAof; - return fullSync; - } - } + public void AddAofSyncTask(AofSyncTaskInfo aofSyncTask) => AofSyncTask = aofSyncTask; /// /// Set status of replica sync session @@ -229,58 +189,102 @@ public async Task WaitForFlush() } } + /// + /// Wait until sync of checkpoint is completed + /// + /// + public async Task WaitForSyncCompletion() + { + while (ssInfo.syncStatus is not SyncStatus.SUCCESS and not SyncStatus.FAILED) + await Task.Yield(); + } + + /// + /// Should stream + /// + /// + public bool NeedToFullSync() + { + // TODO: consolidate disk-based logic + return clusterProvider.serverOptions.ReplicaDisklessSync ? + ShouldStreamDisklessCheckpoint() : true; + + bool ShouldStreamDisklessCheckpoint() + { + var localPrimaryReplId = clusterProvider.replicationManager.PrimaryReplId; + var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal); + sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion; + sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion; + truncatedAof = replicaSyncMetadata.currentAofTailAddress < AofSyncTask.StartAddress; + + // 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 + fullSync = sendMainStore || sendObjectStore || truncatedAof; + return fullSync; + } + } + /// /// Begin syncing AOF to the replica /// public async Task BeginAofSync() { - var aofSyncTask = this.aofSyncTask; - this.aofSyncTask = null; - var mmr = clusterProvider.serverOptions.MainMemoryReplication; - var aofNull = clusterProvider.serverOptions.UseAofNullDevice; - - var canReplayFromAddress = aofSyncTask.StartAddress; - var replicaAofBeginAddress = replicaSyncMetadata.currentAofBeginAddress; - var replicaAofTailAddress = replicaSyncMetadata.currentAofTailAddress; - - var currentAofBeginAddress = canReplayFromAddress; - var currentTailAofAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; - - // TODO: - // If partial sync we need to calculate the beginAddress and endAddress of replica AOF - // to match the primary AOF - if (!fullSync) + try { + var aofSyncTask = AofSyncTask; + var mmr = clusterProvider.serverOptions.MainMemoryReplication; + var aofNull = clusterProvider.serverOptions.UseAofNullDevice; - } + var canReplayFromAddress = aofSyncTask.StartAddress; + var replicaAofBeginAddress = replicaSyncMetadata.currentAofBeginAddress; + var replicaAofTailAddress = replicaSyncMetadata.currentAofTailAddress; + + var currentAofBeginAddress = canReplayFromAddress; + var currentTailAofAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; - var recoverSyncMetadata = new SyncMetadata( - fullSync: fullSync, - originNodeRole: clusterProvider.clusterManager.CurrentConfig.LocalNodeRole, - originNodeId: clusterProvider.clusterManager.CurrentConfig.LocalNodeId, - currentPrimaryReplId: clusterProvider.replicationManager.PrimaryReplId, - currentStoreVersion: currentStoreVersion, - currentObjectStoreVersion: currentObjectStoreVersion, - currentAofBeginAddress: currentAofBeginAddress, - currentAofTailAddress: currentTailAofAddress, - currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset, - checkpointEntry: null); - - var result = await aofSyncTask.garnetClient.ExecuteAttachSync(recoverSyncMetadata.ToByteArray()); - if (!long.TryParse(result, out var syncFromAofAddress)) + // TODO: + // If partial sync we need to calculate the beginAddress and endAddress of replica AOF + // to match the primary AOF + if (!fullSync) + { + + } + + var recoverSyncMetadata = new SyncMetadata( + fullSync: fullSync, + originNodeRole: clusterProvider.clusterManager.CurrentConfig.LocalNodeRole, + originNodeId: clusterProvider.clusterManager.CurrentConfig.LocalNodeId, + currentPrimaryReplId: clusterProvider.replicationManager.PrimaryReplId, + currentStoreVersion: currentStoreVersion, + currentObjectStoreVersion: currentObjectStoreVersion, + currentAofBeginAddress: currentAofBeginAddress, + currentAofTailAddress: currentTailAofAddress, + currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset, + checkpointEntry: null); + + var result = await aofSyncTask.garnetClient.ExecuteAttachSync(recoverSyncMetadata.ToByteArray()); + if (!long.TryParse(result, out var syncFromAofAddress)) + { + logger?.LogError("Failed to parse syncFromAddress at {method}", nameof(BeginAofSync)); + SetStatus(SyncStatus.FAILED, "Failed to parse recovery offset"); + return; + } + + // We have already added the iterator for the covered address above but replica might request an address + // that is ahead of the covered address so we should start streaming from that address in order not to + // introduce duplicate insertions. + if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaSyncMetadata.originNodeId, syncFromAofAddress, out var aofSyncTaskInfo)) + throw new GarnetException("Failed trying to try update replication task"); + if (!clusterProvider.replicationManager.TryConnectToReplica(replicaSyncMetadata.originNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) + throw new GarnetException("Failed connecting to replica for aofSync"); + } + finally { - logger?.LogError("Failed to parse syncFromAddress at {method}", nameof(BeginAofSync)); - SetStatus(SyncStatus.FAILED, "Failed to parse recovery offset"); - return; + AofSyncTask = null; } - - // We have already added the iterator for the covered address above but replica might request an address - // that is ahead of the covered address so we should start streaming from that address in order not to - // introduce duplicate insertions. - if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaSyncMetadata.originNodeId, syncFromAofAddress, out var aofSyncTaskInfo)) - throw new GarnetException("Failed trying to try update replication task"); - if (!clusterProvider.replicationManager.TryConnectToReplica(replicaSyncMetadata.originNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) - throw new GarnetException("Failed connecting to replica for aofSync"); } } } \ 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 a19254704d..ec4db069f3 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -84,7 +84,7 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn } // Wait for main sync task to complete - await replicaSyncSession.CompletePending(); + await replicaSyncSession.WaitForSyncCompletion(); // If session faulted return early if (replicaSyncSession.Failed) @@ -177,7 +177,7 @@ async Task PrepareForSync() // 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 - if (!sessions[i].ShouldStreamCheckpoint()) + if (!sessions[i].NeedToFullSync()) { sessions[i]?.SetStatus(SyncStatus.SUCCESS, "Partial sync"); sessions[i] = null; diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs index e828e1c78a..6dd8eaf131 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs @@ -3,6 +3,9 @@ namespace Garnet.cluster { + /// + /// Replication attach sync status + /// enum SyncStatus : byte { SUCCESS, @@ -11,6 +14,9 @@ enum SyncStatus : byte INITIALIZING } + /// + /// Replication sync status info + /// struct SyncStatusInfo { public SyncStatus syncStatus; diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index 4cb314fa98..4934dc5444 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -46,7 +46,7 @@ internal sealed partial class ReplicaSyncSession( public void Dispose() { - aofSyncTask = null; + AofSyncTask = null; cts.Cancel(); cts.Dispose(); semaphore?.Dispose(); From 288de60c1ee96981b8dd08f5c72813a886f8909d Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 28 Jan 2025 16:18:18 -0800 Subject: [PATCH 14/32] replica fall behind test --- .../DisklessReplication/ReplicaSyncSession.cs | 32 +++--- .../ReplicationSnapshotIterator.cs | 45 +++----- .../ReplicationSyncManager.cs | 106 +++++++++++------- .../ReplicaOps/ReplicaDisklessSync.cs | 7 +- test/Garnet.test.cluster/ClusterTestUtils.cs | 8 ++ ...=> ClusterReplicationDisklessSyncTests.cs} | 66 +++++++++-- 6 files changed, 171 insertions(+), 93 deletions(-) rename test/Garnet.test.cluster/ReplicationTests/{ClusterReplicationDiskless.cs => ClusterReplicationDisklessSyncTests.cs} (55%) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index d1fca08c72..af20f9cfd7 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -16,7 +16,7 @@ internal sealed partial class ReplicaSyncSession Task flushTask; bool sendMainStore = false; bool sendObjectStore = false; - bool truncatedAof = false; + bool outOfRangeAof = false; bool fullSync = false; /// @@ -34,6 +34,11 @@ internal sealed partial class ReplicaSyncSession public long currentObjectStoreVersion; + /// + /// Pessimistic checkpoint covered AOF address + /// + public long checkpointCoveredAofAddress; + /// /// LogError /// @@ -215,14 +220,17 @@ bool ShouldStreamDisklessCheckpoint() var sameHistory = localPrimaryReplId.Equals(replicaSyncMetadata.currentPrimaryReplId, StringComparison.Ordinal); sendMainStore = !sameHistory || replicaSyncMetadata.currentStoreVersion != currentStoreVersion; sendObjectStore = !sameHistory || replicaSyncMetadata.currentObjectStoreVersion != currentObjectStoreVersion; - truncatedAof = replicaSyncMetadata.currentAofTailAddress < AofSyncTask.StartAddress; + + var aofBeginAddress = clusterProvider.storeWrapper.appendOnlyFile.BeginAddress; + var aofTailAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; + outOfRangeAof = replicaSyncMetadata.currentAofTailAddress < aofBeginAddress || replicaSyncMetadata.currentAofTailAddress > aofTailAddress; // 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 - fullSync = sendMainStore || sendObjectStore || truncatedAof; + fullSync = sendMainStore || sendObjectStore || outOfRangeAof; return fullSync; } } @@ -238,20 +246,8 @@ public async Task BeginAofSync() var mmr = clusterProvider.serverOptions.MainMemoryReplication; var aofNull = clusterProvider.serverOptions.UseAofNullDevice; - var canReplayFromAddress = aofSyncTask.StartAddress; - var replicaAofBeginAddress = replicaSyncMetadata.currentAofBeginAddress; - var replicaAofTailAddress = replicaSyncMetadata.currentAofTailAddress; - - var currentAofBeginAddress = canReplayFromAddress; - var currentTailAofAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; - - // TODO: - // If partial sync we need to calculate the beginAddress and endAddress of replica AOF - // to match the primary AOF - if (!fullSync) - { - - } + var currentAofBeginAddress = fullSync ? checkpointCoveredAofAddress : aofSyncTask.StartAddress; + var currentAofTailAddress = clusterProvider.storeWrapper.appendOnlyFile.TailAddress; var recoverSyncMetadata = new SyncMetadata( fullSync: fullSync, @@ -261,7 +257,7 @@ public async Task BeginAofSync() currentStoreVersion: currentStoreVersion, currentObjectStoreVersion: currentObjectStoreVersion, currentAofBeginAddress: currentAofBeginAddress, - currentAofTailAddress: currentTailAofAddress, + currentAofTailAddress: currentAofTailAddress, currentReplicationOffset: clusterProvider.replicationManager.ReplicationOffset, checkpointEntry: null); diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index b182e40dc3..3ed11d6291 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -29,31 +29,21 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ti this.cancellationToken = cancellationToken; this.logger = logger; - var sessionStore = replicationSyncManager.GetSessionStore; - sessions = sessionStore.GetSessions(); - numSessions = sessionStore.GetNumSessions(); + sessions = replicationSyncManager.Sessions; + numSessions = replicationSyncManager.NumSessions; + + var checkpointCoveredAofAddress = replicationSyncManager.ClusterProvider.storeWrapper.appendOnlyFile.TailAddress; + for (var i = 0; i < numSessions; i++) + { + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; + sessions[i].checkpointCoveredAofAddress = checkpointCoveredAofAddress; + } mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); if (!replicationSyncManager.ClusterProvider.serverOptions.DisableObjects) objectStoreSnapshotIterator = new ObjectStoreSnapshotIterator(this); } - public static bool IsActive(ReplicaSyncSession[] session, int offset) - { - // Check if session is null if an error occurred earlier and session was broken - if (session[offset] == null) - return false; - - // Check if connection is still healthy - if (!session[offset].IsConnected) - { - session[offset].SetStatus(SyncStatus.FAILED, "Connection broken"); - session[offset] = null; - return false; - } - return true; - } - public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion, bool isMainStore) { if (cancellationToken.IsCancellationRequested) @@ -61,7 +51,7 @@ public bool OnStart(Guid checkpointToken, long currentVersion, long targetVersio for (var i = 0; i < numSessions; i++) { - if (!IsActive(sessions, i)) continue; + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; sessions[i].InitializeIterationBuffer(); if (isMainStore) sessions[i].currentStoreVersion = targetVersion; @@ -80,8 +70,9 @@ void WaitForFlushAll() // Wait for flush to complete for all and retry to enqueue previous keyValuePair above for (var i = 0; i < numSessions; i++) { - if (!IsActive(sessions, i)) continue; + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; sessions[i].WaitForFlush().GetAwaiter().GetResult(); + if (sessions[i].Failed) sessions[i] = null; } } @@ -96,7 +87,7 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe // Write key value pair to network buffer for (var i = 0; i < numSessions; i++) { - if (!IsActive(sessions, i)) continue; + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; // Initialize header if necessary sessions[i].SetClusterSyncHeader(isMainStore: true); @@ -112,7 +103,7 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe if (!needToFlush) break; // Wait for flush to complete for all and retry to enqueue previous keyValuePair above - WaitForFlushAll(); + replicationSyncManager.WaitForFlush().GetAwaiter().GetResult(); needToFlush = false; } @@ -131,7 +122,7 @@ public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recor // Write key value pair to network buffer for (var i = 0; i < numSessions; i++) { - if (!IsActive(sessions, i)) continue; + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; // Initialize header if necessary sessions[i].SetClusterSyncHeader(isMainStore: false); @@ -147,7 +138,7 @@ public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recor if (!needToFlush) break; // Wait for flush to complete for all and retry to enqueue previous keyValuePair above - WaitForFlushAll(); + replicationSyncManager.WaitForFlush().GetAwaiter().GetResult(); } return true; @@ -158,12 +149,12 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long // Flush remaining data for (var i = 0; i < numSessions; i++) { - if (!IsActive(sessions, i)) continue; + if (!replicationSyncManager.IsActiveSyncSession(i)) continue; sessions[i].SendAndResetIterationBuffer(timeout, cancellationToken); } // Wait for flush and response to complete - WaitForFlushAll(); + replicationSyncManager.WaitForFlush().GetAwaiter().GetResult(); // Enqueue version change commit replicationSyncManager.ClusterProvider.storeWrapper.EnqueueCommit(isMainStore, targetVersion); diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index ec4db069f3..d57c74e1da 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -19,6 +19,10 @@ internal sealed class ReplicationSyncManager public ReplicaSyncSessionTaskStore GetSessionStore { get; } + public int NumSessions { get; private set; } + + public ReplicaSyncSession[] Sessions { get; private set; } + public ClusterProvider ClusterProvider { get; } public ReplicationSyncManager(ClusterProvider clusterProvider, ILogger logger = null) @@ -60,6 +64,32 @@ public bool AddSyncSession(SyncMetadata replicaSyncMetadata, out ReplicaSyncSess } } + public async Task WaitForFlush() + { + for (var i = 0; i < NumSessions; i++) + { + // Wait for network flush + await Sessions[i].WaitForFlush(); + if (Sessions[i].Failed) Sessions[i] = null; + } + } + + public bool IsActiveSyncSession(int offset) + { + // Check if session is null if an error occurred earlier and session was broken + if (Sessions[offset] == null) + return false; + + // Check if connection is still healthy + if (!Sessions[offset].IsConnected) + { + Sessions[offset].SetStatus(SyncStatus.FAILED, "Connection broken"); + Sessions[offset] = null; + return false; + } + return true; + } + /// /// Start sync session /// @@ -94,6 +124,7 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn return replicaSyncSession.GetSyncStatusInfo; } + // Start AOF sync background task for this replica await replicaSyncSession.BeginAofSync(); return replicaSyncSession.GetSyncStatusInfo; @@ -111,9 +142,8 @@ async Task StreamingSnapshotSync() var disklessRepl = ClusterProvider.serverOptions.ReplicaDisklessSync; var disableObjects = ClusterProvider.serverOptions.DisableObjects; - // Replica sync session - var numSessions = GetSessionStore.GetNumSessions(); - var sessions = GetSessionStore.GetSessions(); + NumSessions = GetSessionStore.GetNumSessions(); + Sessions = GetSessionStore.GetSessions(); try { @@ -125,13 +155,30 @@ async Task StreamingSnapshotSync() await PrepareForSync(); // Stream checkpoint to replicas - await StreamDisklessCheckpoint(); + await TakeStreamingCheckpoint(); + + // Stream Diskless + async Task TakeStreamingCheckpoint() + { + // Main snapshot iterator manager + var manager = new SnapshotIteratorManager(this, clusterTimeout, cts.Token, logger); + + // Iterate through main store + var mainStoreResult = await ClusterProvider.storeWrapper.store. + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + + if (!ClusterProvider.serverOptions.DisableObjects) + { + // Iterate through object store + var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + } + } } finally { // Notify sync session of success success - for (var i = 0; i < numSessions; i++) - sessions[i]?.SetStatus(SyncStatus.SUCCESS); + for (var i = 0; i < NumSessions; i++) + Sessions[i]?.SetStatus(SyncStatus.SUCCESS); // Clear array of sync sessions GetSessionStore.Clear(); @@ -151,11 +198,11 @@ async Task PrepareForSync() #region pauseAofTruncation while (true) { - // Calculate minimum address from which replicas should start streaming from - var syncFromAddress = ClusterProvider.storeWrapper.appendOnlyFile.TailAddress; + // Minimum address that we can serve assuming aof-locking and no aof-null-device + var minServiceableAofAddress = ClusterProvider.storeWrapper.appendOnlyFile.BeginAddress; // Lock AOF address for sync streaming - if (ClusterProvider.replicationManager.TryAddReplicationTasks(GetSessionStore.GetSessions(), syncFromAddress)) + if (ClusterProvider.replicationManager.TryAddReplicationTasks(GetSessionStore.GetSessions(), minServiceableAofAddress)) break; // Retry if failed to lock AOF address because truncation occurred @@ -164,58 +211,41 @@ async Task PrepareForSync() #endregion #region initializeConnection - for (var i = 0; i < numSessions; i++) + for (var i = 0; i < NumSessions; i++) { try { // Initialize connections - sessions[i].Connect(); + Sessions[i].Connect(); // Set store version to operate on - sessions[i].currentStoreVersion = ClusterProvider.storeWrapper.store.CurrentVersion; - sessions[i].currentObjectStoreVersion = disableObjects ? -1 : ClusterProvider.storeWrapper.objectStore.CurrentVersion; + 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 - if (!sessions[i].NeedToFullSync()) + if (!Sessions[i].NeedToFullSync()) { - sessions[i]?.SetStatus(SyncStatus.SUCCESS, "Partial sync"); - sessions[i] = null; + Sessions[i]?.SetStatus(SyncStatus.SUCCESS, "Partial sync"); + Sessions[i] = null; } else { // Reset replica database in preparation for full sync - sessions[i].SetFlushTask(sessions[i].ExecuteAsync(["FLUSHALL"]), timeout: clusterTimeout, cts.Token); - await sessions[i].WaitForFlush(); - if (sessions[i].Failed) sessions[i] = null; + Sessions[i].SetFlushTask(Sessions[i].ExecuteAsync(["FLUSHALL"]), timeout: clusterTimeout, cts.Token); } } catch (Exception ex) { - sessions[i]?.SetStatus(SyncStatus.FAILED, ex.Message); - sessions[i] = null; + Sessions[i]?.SetStatus(SyncStatus.FAILED, ex.Message); + Sessions[i] = null; } - } + + await WaitForFlush(); #endregion } } } - - async Task StreamDisklessCheckpoint() - { - // Main snapshot iterator manager - var manager = new SnapshotIteratorManager(this, clusterTimeout, cts.Token, logger); - - // Iterate through main store - var mainStoreResult = await ClusterProvider.storeWrapper.store. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); - - if (!ClusterProvider.serverOptions.DisableObjects) - { - // Iterate through object store - var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); - } - } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 51742a7293..165ecce0fe 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -122,6 +122,7 @@ async Task TryBeginReplicaSync() } finally { + replicateLock.WriteUnlock(); gcs?.Dispose(); recvCheckpointHandler?.Dispose(); } @@ -140,10 +141,12 @@ public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata) if (!primarySyncMetadata.fullSync) { - // TODO: replay local AOF + // For diskless replication if we are performing a partial sync need to start streaming from replicationOffset + // hence our tail needs to be reset to that point + aofTailAddress = replicationOffset = ReplicationOffset; } - storeWrapper.appendOnlyFile.Initialize(aofBeginAddress, aofBeginAddress); + storeWrapper.appendOnlyFile.Initialize(aofBeginAddress, aofTailAddress); // Set DB version storeWrapper.store.SetVersion(primarySyncMetadata.currentStoreVersion); diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index bad107c5e4..1e8afee89f 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -1276,6 +1276,14 @@ public void WaitAll(ILogger logger) } } + public bool IsKnown(int nodeIndex, int knownNodeIndex, ILogger logger = null) + { + var toKnowNodeId = ClusterNodes(knownNodeIndex).Nodes.First().NodeId; + var nodeConfig = ClusterNodes(nodeIndex); + + return nodeConfig.Nodes.Any(x => x.NodeId.Equals(toKnowNodeId, StringComparison.OrdinalIgnoreCase)); + } + public void WaitUntilNodeIsKnown(int nodeIndex, int toKnowNode, ILogger logger = null) { var toKnowNodeId = ClusterNodes(toKnowNode).Nodes.First().NodeId; diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs similarity index 55% rename from test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs rename to test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index d2a29fcf84..448545e356 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDiskless.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -16,7 +16,8 @@ namespace Garnet.test.cluster /// 3. Replica same history and different version and AOF /// 4. Replica different history, version and AOF /// - public class ClusterReplicationDiskless + [NonParallelizable] + public class ClusterReplicationDisklessSyncTests { ClusterTestContext context; readonly int keyCount = 256; @@ -24,10 +25,7 @@ public class ClusterReplicationDiskless protected bool useTLS = false; protected bool asyncReplay = false; - public Dictionary monitorTests = new() - { - {"ClusterReplicationSimpleFailover", LogLevel.Warning}, - }; + public Dictionary monitorTests = []; [SetUp] public virtual void Setup() @@ -44,8 +42,8 @@ public virtual void TearDown() void PopulatePrimary(int primaryIndex, bool disableObjects, bool performRMW) { - context.kvPairs = []; - context.kvPairsObj = []; + context.kvPairs = context.kvPairs ?? ([]); + context.kvPairsObj = context.kvPairsObj ?? ([]); var addCount = 5; var keyLength = 16; var kvpairCount = keyCount; @@ -71,7 +69,7 @@ void Validate(int primaryIndex, int replicaIndex, bool disableObjects) [Test, Order(1)] [Category("REPLICATION")] - public void ClusterSimpleAttachDisklessSync([Values] bool disableObjects, [Values] bool performRMW) + public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Values] bool performRMW) { var nodes_count = 2; var primaryIndex = 0; @@ -100,5 +98,57 @@ public void ClusterSimpleAttachDisklessSync([Values] bool disableObjects, [Value // Validate replica data Validate(primaryIndex, replicaIndex, disableObjects); } + + + [Test, Order(1)] + [Category("REPLICATION")] + public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] bool performRMW) + { + var nodes_count = 2; + var primaryIndex = 0; + var replicaIndex = 1; + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateConnection(useTLS: useTLS); + + // Setup primary and introduce it to future replica + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaIndex, replicaIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaIndex, logger: context.logger); + + // Ensure node is known + context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); + + // Attach sync session + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Wait for replica to catch up + context.clusterTestUtils.WaitForReplicaAofSync(primaryIndex, replicaIndex, logger: context.logger); + + // Populate Primary + PopulatePrimary(primaryIndex, disableObjects, performRMW); + + // Validate replica data + Validate(primaryIndex, replicaIndex, disableObjects); + + // Soft reset replica + _ = context.clusterTestUtils.ClusterReset(replicaIndex, soft: true, expiry: 1, logger: context.logger); + context.clusterTestUtils.BumpEpoch(replicaIndex, logger: context.logger); + // Re-introduce node after reset + while (!context.clusterTestUtils.IsKnown(replicaIndex, primaryIndex, logger: context.logger)) + { + ClusterTestUtils.BackOff(cancellationToken: context.cts.Token); + context.clusterTestUtils.Meet(replicaIndex, primaryIndex, logger: context.logger); + } + + // Populate Primary (ahead of replica) + PopulatePrimary(primaryIndex, disableObjects, performRMW); + + // Re-attach sync session + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Validate replica data + Validate(primaryIndex, replicaIndex, disableObjects); + } } } \ No newline at end of file From 99f9ce602ad2251fe4427ba7f8d68019cbb7280f Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 29 Jan 2025 10:39:03 -0800 Subject: [PATCH 15/32] wip --- .../ClusterReplicationDisklessSyncTests.cs | 30 ++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 448545e356..cd7d571de9 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -67,6 +67,11 @@ void Validate(int primaryIndex, int replicaIndex, bool disableObjects) context.ValidateNodeObjects(ref context.kvPairsObj, replicaIndex); } + /// + /// Attach empty replica after primary has been populated with some data + /// + /// + /// [Test, Order(1)] [Category("REPLICATION")] public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Values] bool performRMW) @@ -100,7 +105,13 @@ public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Value } - [Test, Order(1)] + /// + /// Re-attach replica on disconnect after it has received some amount of data. + /// The replica should replay only the portion it missed without doing a full sync + /// + /// + /// + [Test, Order(2)] [Category("REPLICATION")] public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] bool performRMW) { @@ -150,5 +161,22 @@ public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] // Validate replica data Validate(primaryIndex, replicaIndex, disableObjects); } + + /// + /// Attach one replica and populate it with data through primary. + /// Disconnect replica and attach a new empty replica. + /// Populate new replica with more data + /// Re-attach disconnected replica. + /// This should perform a full sync when the old replica is attach because primary will be in version v + /// (because of syncing with new empty replica) and old replica will be in version v - 1. + /// + /// + /// + [Test, Order(3)] + [Category("REPLICATION")] + public void ClusterDBVersionAligmentDisklessSync([Values] bool disableObjects, [Values] bool performRMW) + { + + } } } \ No newline at end of file From 2dadafe16921620400827b8ccd0c5384ae44be6b Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 29 Jan 2025 14:17:56 -0800 Subject: [PATCH 16/32] register cts at wait for sync completion --- .../DisklessReplication/ReplicaSyncSession.cs | 24 ++++++++++++------- .../ReplicationSnapshotIterator.cs | 9 ++++--- .../ReplicationSyncManager.cs | 10 +++++--- .../PrimaryOps/ReplicaSyncSession.cs | 4 ++++ .../PrimaryOps/ReplicaSyncSessionTaskStore.cs | 2 +- .../ReplicaOps/ReplicaDisklessSync.cs | 4 ++-- 6 files changed, 34 insertions(+), 19 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index af20f9cfd7..d550bda964 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System; -using System.Threading; using System.Threading.Tasks; using Garnet.common; using Microsoft.Extensions.Logging; @@ -128,10 +127,10 @@ public bool TryWriteKeyValueByteArray(byte[] key, byte[] value, long expiration, /// Send and reset iteration buffer /// /// - public void SendAndResetIterationBuffer(TimeSpan timeout, CancellationToken token) + public void SendAndResetIterationBuffer() { WaitForFlush().GetAwaiter().GetResult(); - SetFlushTask(AofSyncTask.garnetClient.SendAndResetIterationBuffer(), timeout: timeout, token: token); + SetFlushTask(AofSyncTask.garnetClient.SendAndResetIterationBuffer()); } #endregion @@ -157,9 +156,7 @@ public void SetStatus(SyncStatus status, string error = null) /// Set network flush task for checkpoint snapshot stream data /// /// - /// - /// - public void SetFlushTask(Task task, TimeSpan timeout, CancellationToken token) + public void SetFlushTask(Task task) { if (task != null) { @@ -200,8 +197,19 @@ public async Task WaitForFlush() /// public async Task WaitForSyncCompletion() { - while (ssInfo.syncStatus is not SyncStatus.SUCCESS and not SyncStatus.FAILED) - await Task.Yield(); + try + { + while (ssInfo.syncStatus is not SyncStatus.SUCCESS and not SyncStatus.FAILED) + { + token.ThrowIfCancellationRequested(); + await Task.Yield(); + } + } + catch (Exception ex) + { + logger?.LogError(ex, "{method} failed waiting for sync", nameof(WaitForSyncCompletion)); + SetStatus(SyncStatus.FAILED, "Wait for sync task faulted"); + } } /// diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index 3ed11d6291..31c8c38ad7 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -22,10 +22,9 @@ internal sealed unsafe class SnapshotIteratorManager readonly ReplicaSyncSession[] sessions; readonly int numSessions; - public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, TimeSpan timeout, CancellationToken cancellationToken, ILogger logger = null) + public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, CancellationToken cancellationToken, ILogger logger = null) { this.replicationSyncManager = replicationSyncManager; - this.timeout = timeout; this.cancellationToken = cancellationToken; this.logger = logger; @@ -95,7 +94,7 @@ public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMe // Try to write to network buffer. If failed we need to retry if (!sessions[i].TryWriteKeyValueSpanByte(ref key, ref value, out var task)) { - sessions[i].SetFlushTask(task, timeout, cancellationToken); + sessions[i].SetFlushTask(task); needToFlush = true; } } @@ -130,7 +129,7 @@ public bool Reader(ref byte[] key, ref IGarnetObject value, RecordMetadata recor // Try to write to network buffer. If failed we need to retry if (!sessions[i].TryWriteKeyValueByteArray(key, objectData, value.Expiration, out var task)) { - sessions[i].SetFlushTask(task, timeout, cancellationToken); + sessions[i].SetFlushTask(task); needToFlush = true; } } @@ -150,7 +149,7 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long for (var i = 0; i < numSessions; i++) { if (!replicationSyncManager.IsActiveSyncSession(i)) continue; - sessions[i].SendAndResetIterationBuffer(timeout, cancellationToken); + sessions[i].SendAndResetIterationBuffer(); } // Wait for flush and response to complete diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index d57c74e1da..552d901bda 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -51,7 +51,7 @@ public void Dispose() /// public bool AddSyncSession(SyncMetadata replicaSyncMetadata, out ReplicaSyncSession replicaSyncSession) { - replicaSyncSession = new ReplicaSyncSession(ClusterProvider.storeWrapper, ClusterProvider, replicaSyncMetadata, logger: logger); + replicaSyncSession = new ReplicaSyncSession(ClusterProvider.storeWrapper, ClusterProvider, replicaSyncMetadata, clusterTimeout, cts.Token, logger: logger); replicaSyncSession.SetStatus(SyncStatus.INITIALIZING); try { @@ -161,7 +161,7 @@ async Task StreamingSnapshotSync() async Task TakeStreamingCheckpoint() { // Main snapshot iterator manager - var manager = new SnapshotIteratorManager(this, clusterTimeout, cts.Token, logger); + var manager = new SnapshotIteratorManager(this, cts.Token, logger); // Iterate through main store var mainStoreResult = await ClusterProvider.storeWrapper.store. @@ -174,6 +174,10 @@ async Task TakeStreamingCheckpoint() } } } + catch (Exception ex) + { + logger?.LogError(ex, "{method} faulted", nameof(StreamingSnapshotSync)); + } finally { // Notify sync session of success success @@ -232,7 +236,7 @@ async Task PrepareForSync() else { // Reset replica database in preparation for full sync - Sessions[i].SetFlushTask(Sessions[i].ExecuteAsync(["FLUSHALL"]), timeout: clusterTimeout, cts.Token); + Sessions[i].SetFlushTask(Sessions[i].ExecuteAsync(["FLUSHALL"])); } } catch (Exception ex) diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index 4934dc5444..f6e2729086 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -19,6 +19,8 @@ internal sealed partial class ReplicaSyncSession( StoreWrapper storeWrapper, ClusterProvider clusterProvider, SyncMetadata replicaSyncMetadata = null, + TimeSpan timeout = default, + CancellationToken token = default, string replicaNodeId = null, string replicaAssignedPrimaryId = null, CheckpointEntry replicaCheckpointEntry = null, @@ -29,6 +31,8 @@ internal sealed partial class ReplicaSyncSession( readonly StoreWrapper storeWrapper = storeWrapper; readonly ClusterProvider clusterProvider = clusterProvider; public readonly SyncMetadata replicaSyncMetadata = replicaSyncMetadata; + readonly TimeSpan timeout = timeout; + readonly CancellationToken token = token; readonly CancellationTokenSource cts = new(); SectorAlignedBufferPool bufferPool = null; readonly SemaphoreSlim semaphore = new(0); diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs index 682433cf14..fb6d288b0e 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs @@ -80,7 +80,7 @@ public bool TryAddReplicaSyncSession(ReplicaSyncSession session) public bool TryAddReplicaSyncSession(string replicaNodeId, string replicaAssignedPrimaryId, CheckpointEntry replicaCheckpointEntry, long replicaAofBeginAddress, long replicaAofTailAddress) { - var retSession = new ReplicaSyncSession(storeWrapper, clusterProvider, replicaSyncMetadata: null, replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress, logger); + var retSession = new ReplicaSyncSession(storeWrapper, clusterProvider, replicaSyncMetadata: null, timeout: default, token: default, replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress, logger); var success = false; try { diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 165ecce0fe..77cde55e0b 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -48,7 +48,7 @@ public bool TryReplicateDisklessSync( async Task TryBeginReplicaSync() { var disklessSync = clusterProvider.serverOptions.ReplicaDisklessSync; - var dissableObjects = clusterProvider.serverOptions.DisableObjects; + var disableObjects = clusterProvider.serverOptions.DisableObjects; GarnetClientSession gcs = null; try { @@ -106,7 +106,7 @@ async Task TryBeginReplicaSync() originNodeId: current.LocalNodeId, currentPrimaryReplId: PrimaryReplId, currentStoreVersion: storeWrapper.store.CurrentVersion, - currentObjectStoreVersion: dissableObjects ? -1 : storeWrapper.objectStore.CurrentVersion, + currentObjectStoreVersion: disableObjects ? -1 : storeWrapper.objectStore.CurrentVersion, currentAofBeginAddress: storeWrapper.appendOnlyFile.BeginAddress, currentAofTailAddress: storeWrapper.appendOnlyFile.TailAddress, currentReplicationOffset: ReplicationOffset, From 670b9959b8542fffff4880da1a979896c75066be Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 29 Jan 2025 14:58:07 -0800 Subject: [PATCH 17/32] add db version alignment test --- test/Garnet.test.cluster/ClusterTestUtils.cs | 48 ++++++++++ .../ClusterReplicationDisklessSyncTests.cs | 92 +++++++++++++++++-- 2 files changed, 130 insertions(+), 10 deletions(-) diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index 1e8afee89f..ed14d9d210 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -100,6 +100,13 @@ public enum ReplicationInfoItem : byte PRIMARY_FAILOVER_STATE, } + public enum StoreInfoItem + { + CurrentVersion, + LastCheckpointedVersion, + RecoveredVersion + } + public static class EndpointExtensions { public static IPEndPoint ToIPEndPoint(this EndPoint endPoint) @@ -2694,6 +2701,47 @@ public string GetFailoverState(IPEndPoint endPoint, ILogger logger = null) return items; } + public int GetStoreCurrentVersion(int nodeIndex, bool isMainStore, ILogger logger = null) + { + var result = GetStoreInfo(endpoints[nodeIndex].ToIPEndPoint(), [StoreInfoItem.CurrentVersion], isMainStore, logger); + ClassicAssert.AreEqual(1, result.Count); + return int.Parse(result[0].Item2); + } + + public List<(StoreInfoItem, string)> GetStoreInfo(int nodeIndex, HashSet infoItems, bool isMainStore, ILogger logger = null) + => GetStoreInfo(endpoints[nodeIndex].ToIPEndPoint(), infoItems, isMainStore, logger); + + private List<(StoreInfoItem, string)> GetStoreInfo(IPEndPoint endPoint, HashSet infoItems, bool isMainStore, ILogger logger = null) + { + var fields = new List<(StoreInfoItem, string)>(); + try + { + var server = redis.GetServer(endPoint); + var result = server.InfoRawAsync(isMainStore ? "store" : "objectstore").Result; + var data = result.Split('\n'); + foreach (var line in data) + { + if (line.StartsWith('#')) + continue; + var field = line.Trim().Split(':'); + + if (!Enum.TryParse(field[0], ignoreCase: true, out StoreInfoItem type)) + continue; + + if (infoItems.Contains(type)) + fields.Add((type, field[1])); + } + } + catch (Exception ex) + { + logger?.LogError(ex, "An error has occurred; GetReplicationInfo"); + Assert.Fail(ex.Message); + } + + return fields; + } + + public string GetInfo(int nodeIndex, string section, string segment, ILogger logger = null) => GetInfo(endpoints[nodeIndex].ToIPEndPoint(), section, segment, logger); diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index cd7d571de9..38ee67d5b1 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -4,6 +4,7 @@ using System.Collections.Generic; using Microsoft.Extensions.Logging; using NUnit.Framework; +using NUnit.Framework.Legacy; namespace Garnet.test.cluster { @@ -67,6 +68,19 @@ void Validate(int primaryIndex, int replicaIndex, bool disableObjects) context.ValidateNodeObjects(ref context.kvPairsObj, replicaIndex); } + void ResetAndReAttach(int replicaIndex, int primaryIndex, bool soft) + { + // Soft reset replica + _ = context.clusterTestUtils.ClusterReset(replicaIndex, soft: soft, expiry: 1, logger: context.logger); + context.clusterTestUtils.BumpEpoch(replicaIndex, logger: context.logger); + // Re-introduce node after reset + while (!context.clusterTestUtils.IsKnown(replicaIndex, primaryIndex, logger: context.logger)) + { + ClusterTestUtils.BackOff(cancellationToken: context.cts.Token); + context.clusterTestUtils.Meet(replicaIndex, primaryIndex, logger: context.logger); + } + } + /// /// Attach empty replica after primary has been populated with some data /// @@ -142,15 +156,8 @@ public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] // Validate replica data Validate(primaryIndex, replicaIndex, disableObjects); - // Soft reset replica - _ = context.clusterTestUtils.ClusterReset(replicaIndex, soft: true, expiry: 1, logger: context.logger); - context.clusterTestUtils.BumpEpoch(replicaIndex, logger: context.logger); - // Re-introduce node after reset - while (!context.clusterTestUtils.IsKnown(replicaIndex, primaryIndex, logger: context.logger)) - { - ClusterTestUtils.BackOff(cancellationToken: context.cts.Token); - context.clusterTestUtils.Meet(replicaIndex, primaryIndex, logger: context.logger); - } + // Reset and re-attach replica as primary + ResetAndReAttach(replicaIndex, primaryIndex, soft: true); // Populate Primary (ahead of replica) PopulatePrimary(primaryIndex, disableObjects, performRMW); @@ -174,9 +181,74 @@ public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] /// [Test, Order(3)] [Category("REPLICATION")] - public void ClusterDBVersionAligmentDisklessSync([Values] bool disableObjects, [Values] bool performRMW) + public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, [Values] bool performRMW) { + var nodes_count = 3; + var primaryIndex = 0; + var replicaOneIndex = 1; + var replicaTwoIndex = 2; + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateConnection(useTLS: useTLS); + + // Setup primary and introduce it to future replica + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaOneIndex, replicaOneIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaTwoIndex, replicaTwoIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaOneIndex, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaTwoIndex, logger: context.logger); + + // Ensure node everybody knowns everybody + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaOneIndex, primaryIndex, logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaTwoIndex, primaryIndex, logger: context.logger); + + // Populate Primary + PopulatePrimary(primaryIndex, disableObjects, performRMW); + + // Attach first replica + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaOneIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Validate first replica data + Validate(primaryIndex, replicaOneIndex, disableObjects); + + // Validate db version + var primaryVersion = context.clusterTestUtils.GetStoreCurrentVersion(primaryIndex, isMainStore: true, logger: context.logger); + var replicaOneVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaOneIndex, isMainStore: true, logger: context.logger); + ClassicAssert.AreEqual(2, primaryVersion); + ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); + + // Reset and re-attach replica as primary + ResetAndReAttach(replicaOneIndex, primaryIndex, soft: true); + + // Attach second replica + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaTwoIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Populate primary with more data + PopulatePrimary(primaryIndex, disableObjects, performRMW); + // Validate second replica data + Validate(primaryIndex, replicaTwoIndex, disableObjects); + + // Validate db version + primaryVersion = context.clusterTestUtils.GetStoreCurrentVersion(primaryIndex, isMainStore: true, logger: context.logger); + replicaOneVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaOneIndex, isMainStore: true, logger: context.logger); + var replicaTwoVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaTwoIndex, isMainStore: true, logger: context.logger); + ClassicAssert.AreEqual(3, primaryVersion); + ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); + ClassicAssert.AreEqual(2, replicaOneVersion); + + // Re-attach first replica + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaOneIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Validate second replica data + Validate(primaryIndex, replicaOneIndex, disableObjects); + + primaryVersion = context.clusterTestUtils.GetStoreCurrentVersion(primaryIndex, isMainStore: true, logger: context.logger); + replicaOneVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaOneIndex, isMainStore: true, logger: context.logger); + replicaTwoVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaTwoIndex, isMainStore: true, logger: context.logger); + ClassicAssert.AreEqual(4, primaryVersion); + ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); + ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); } } } \ No newline at end of file From 51d216840670391c6cb574f22b35aac3ee494e14 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 30 Jan 2025 13:23:57 -0800 Subject: [PATCH 18/32] avoid using close lock for leader based syncing --- .../ReplicationSyncManager.cs | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 552d901bda..5b99337035 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -106,8 +106,9 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn // Started syncing replicaSyncSession.SetStatus(SyncStatus.INPROGRESS); + var isLeader = GetSessionStore.GetSessions()[0] == replicaSyncSession; // Only one thread will acquire this lock - if (syncInProgress.OneWriteLock()) + if (isLeader) { // Launch a background task to sync the attached replicas using streaming snapshot _ = Task.Run(() => StreamingSnapshotSync()); @@ -147,6 +148,9 @@ async Task StreamingSnapshotSync() try { + // Lock to avoid the addition of new replica sync sessions while sync is in progress + syncInProgress.WriteLock(); + // Take lock to ensure no other task will be taking a checkpoint while (!ClusterProvider.storeWrapper.TryPauseCheckpoints()) await Task.Yield(); @@ -173,17 +177,19 @@ async Task TakeStreamingCheckpoint() var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); } } + + // Notify sync session of success success + for (var i = 0; i < NumSessions; i++) + Sessions[i]?.SetStatus(SyncStatus.SUCCESS); } catch (Exception ex) { logger?.LogError(ex, "{method} faulted", nameof(StreamingSnapshotSync)); + for (var i = 0; i < NumSessions; i++) + Sessions[i]?.SetStatus(SyncStatus.FAILED, ex.Message); } finally { - // Notify sync session of success success - for (var i = 0; i < NumSessions; i++) - Sessions[i]?.SetStatus(SyncStatus.SUCCESS); - // Clear array of sync sessions GetSessionStore.Clear(); From 8df1dd2f4c90897eceef256b36cbfb520ca83bd4 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 30 Jan 2025 17:26:49 -0800 Subject: [PATCH 19/32] truncate AOF after streaming checkpoint is taken --- .../DisklessReplication/ReplicationSnapshotIterator.cs | 6 ++++-- .../DisklessReplication/ReplicationSyncManager.cs | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index 31c8c38ad7..e9813fd8b6 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -22,6 +22,8 @@ internal sealed unsafe class SnapshotIteratorManager readonly ReplicaSyncSession[] sessions; readonly int numSessions; + public long CheckpointCoveredAddress { get; private set; } + public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, CancellationToken cancellationToken, ILogger logger = null) { this.replicationSyncManager = replicationSyncManager; @@ -31,11 +33,11 @@ public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, Ca sessions = replicationSyncManager.Sessions; numSessions = replicationSyncManager.NumSessions; - var checkpointCoveredAofAddress = replicationSyncManager.ClusterProvider.storeWrapper.appendOnlyFile.TailAddress; + CheckpointCoveredAddress = replicationSyncManager.ClusterProvider.storeWrapper.appendOnlyFile.TailAddress; for (var i = 0; i < numSessions; i++) { if (!replicationSyncManager.IsActiveSyncSession(i)) continue; - sessions[i].checkpointCoveredAofAddress = checkpointCoveredAofAddress; + sessions[i].checkpointCoveredAofAddress = CheckpointCoveredAddress; } mainStoreSnapshotIterator = new MainStoreSnapshotIterator(this); diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 5b99337035..a7887ac523 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -176,6 +176,8 @@ async Task TakeStreamingCheckpoint() // Iterate through object store var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); } + + ClusterProvider.replicationManager.SafeTruncateAof(manager.CheckpointCoveredAddress); } // Notify sync session of success success From e3ef1b1c06fcc022ffc97f33be97d6a34f8d7d36 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 30 Jan 2025 16:51:48 -0800 Subject: [PATCH 20/32] add tests for failover with diskless replication --- libs/cluster/Session/ReplicaOfCommand.cs | 6 +- .../Garnet.test.cluster/ClusterConfigTests.cs | 8 +- .../Garnet.test.cluster/ClusterTestContext.cs | 6 +- test/Garnet.test.cluster/ClusterTestUtils.cs | 35 +++-- .../ClusterReplicationDisklessSyncTests.cs | 129 +++++++++++++++++- 5 files changed, 163 insertions(+), 21 deletions(-) diff --git a/libs/cluster/Session/ReplicaOfCommand.cs b/libs/cluster/Session/ReplicaOfCommand.cs index d654f3bd8b..cda71c55cf 100644 --- a/libs/cluster/Session/ReplicaOfCommand.cs +++ b/libs/cluster/Session/ReplicaOfCommand.cs @@ -67,7 +67,11 @@ private bool TryREPLICAOF(out bool invalidParameters) return true; } - if (!clusterProvider.replicationManager.TryBeginReplicate(this, primaryId, background: false, force: true, out var errorMessage)) + var success = clusterProvider.serverOptions.ReplicaDisklessSync ? + clusterProvider.replicationManager.TryReplicateDisklessSync(this, primaryId, background: false, force: true, out var errorMessage) : + clusterProvider.replicationManager.TryBeginReplicate(this, primaryId, background: false, force: true, out errorMessage); + + if (!success) { while (!RespWriteUtils.TryWriteError(errorMessage, ref dcurr, dend)) SendAndReset(); diff --git a/test/Garnet.test.cluster/ClusterConfigTests.cs b/test/Garnet.test.cluster/ClusterConfigTests.cs index ade91dc42e..9d062437de 100644 --- a/test/Garnet.test.cluster/ClusterConfigTests.cs +++ b/test/Garnet.test.cluster/ClusterConfigTests.cs @@ -37,19 +37,19 @@ public void TearDown() [Category("CLUSTER-CONFIG"), CancelAfter(1000)] public void ClusterConfigInitializesUnassignedWorkerTest() { - ClusterConfig config = new ClusterConfig().InitializeLocalWorker( + var config = new ClusterConfig().InitializeLocalWorker( Generator.CreateHexId(), "127.0.0.1", 7001, configEpoch: 0, - NodeRole.PRIMARY, + Garnet.cluster.NodeRole.PRIMARY, null, ""); (string address, int port) = config.GetWorkerAddress(0); Assert.That(address == "unassigned"); Assert.That(port == 0); - Assert.That(NodeRole.UNASSIGNED == config.GetNodeRoleFromNodeId("asdasdqwe")); + Assert.That(Garnet.cluster.NodeRole.UNASSIGNED == config.GetNodeRoleFromNodeId("asdasdqwe")); var configBytes = config.ToByteArray(); var restoredConfig = ClusterConfig.FromByteArray(configBytes); @@ -57,7 +57,7 @@ public void ClusterConfigInitializesUnassignedWorkerTest() (address, port) = restoredConfig.GetWorkerAddress(0); Assert.That(address == "unassigned"); Assert.That(port == 0); - Assert.That(NodeRole.UNASSIGNED == restoredConfig.GetNodeRoleFromNodeId("asdasdqwe")); + Assert.That(Garnet.cluster.NodeRole.UNASSIGNED == restoredConfig.GetNodeRoleFromNodeId("asdasdqwe")); } [Test, Order(2)] diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 40a8abd828..ef64cbf02f 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -401,9 +401,11 @@ public void PopulatePrimaryWithObjects(ref Dictionary> kvPairs while (kvPairsObj.ContainsKey(key)) key = clusterTestUtils.RandomStr(keyLength); kvPairsObj.Add(key, value); - + int count; if (!set) - clusterTestUtils.Lpush(primaryIndex, key, value, logger); + { + count = clusterTestUtils.Lpush(primaryIndex, key, value, logger); + } else clusterTestUtils.Sadd(primaryIndex, key, value, logger); diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index ed14d9d210..93b8380cf0 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -12,7 +12,6 @@ using System.Threading; using System.Threading.Tasks; using Garnet.client; -using Garnet.cluster; using Garnet.common; using GarnetClusterManagement; using Microsoft.Extensions.Logging; @@ -63,7 +62,7 @@ public struct SlotItem public NodeNetInfo[] nnInfo; } - public enum Role + public enum NodeRole { PRIMARY, REPLICA @@ -75,7 +74,7 @@ public struct NodeInfo public string nodeid; public string address; public int port; - public Role role; + public NodeRole role; public long replicationOffset; } @@ -287,7 +286,7 @@ private ClientClusterConfig GetClusterConfig(int primary_count, int node_count, endpoints[i].Address.ToString(), endpoints[i].Port, i + 1 + (!isPrimary ? 1 : 0), - isPrimary ? NodeRole.PRIMARY : NodeRole.REPLICA, + isPrimary ? Garnet.cluster.NodeRole.PRIMARY : Garnet.cluster.NodeRole.REPLICA, primaryId, hostname, isPrimary && i < slotRanges.Length ? slotRanges[i] : null); @@ -386,7 +385,7 @@ private void WaitForSync(ClientClusterConfig clusterConfig) nodeid = GetNodeIdFromNode(i, logger), address = endpoint.Address.ToString(), port = endpoint.Port, - role = Role.PRIMARY, + role = NodeRole.PRIMARY, replicationOffset = 0 } } @@ -450,7 +449,7 @@ private void WaitForSync(ClientClusterConfig clusterConfig) nodeid = GetNodeIdFromNode(i, logger), address = GetAddressFromNodeIndex(i), port = GetPortFromNodeIndex(i), - role = Role.REPLICA, + role = NodeRole.REPLICA, replicationOffset = 0 } ); @@ -2120,7 +2119,7 @@ public List ClusterShards(IPEndPoint endPoint, ILogger logger = null) nodeid = (string)node[1], port = (int)node[3], address = (string)node[5], - role = Enum.Parse((string)node[7]), + role = Enum.Parse((string)node[7]), replicationOffset = (long)node[9] }; shardInfo.nodes.Add(nodeInfo); @@ -2385,7 +2384,7 @@ public string GetMultiKey(int nodeIndex, List keys, out List get } } - public void Lpush(int nodeIndex, string key, List elements, ILogger logger = null) + public int Lpush(int nodeIndex, string key, List elements, ILogger logger = null) { try { @@ -2396,11 +2395,13 @@ public void Lpush(int nodeIndex, string key, List elements, ILogger logger var result = (int)server.Execute("LPUSH", args); ClassicAssert.AreEqual(elements.Count, result); + return result; } catch (Exception ex) { logger?.LogError(ex, "lpush error"); Assert.Fail(ex.Message); + return -1; } } @@ -2549,6 +2550,24 @@ public long GetConnectedReplicas(IPEndPoint endPoint, ILogger logger = null) } } + public Role RoleCommand(int nodeIndex, ILogger logger = null) + => RoleCommand(endpoints[nodeIndex].ToIPEndPoint(), logger); + + public Role RoleCommand(IPEndPoint endPoint, ILogger logger = null) + { + try + { + var server = redis.GetServer(endPoint); + return server.Role(); + } + catch (Exception ex) + { + logger?.LogError(ex, "{command}", nameof(NodeRole)); + Assert.Fail(ex.Message); + return null; + } + } + public string GetReplicationRole(int nodeIndex, ILogger logger = null) => GetReplicationRole((IPEndPoint)endpoints[nodeIndex], logger); diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 38ee67d5b1..ca58f2d4a8 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -48,8 +48,6 @@ void PopulatePrimary(int primaryIndex, bool disableObjects, bool performRMW) var addCount = 5; var keyLength = 16; var kvpairCount = keyCount; - context.kvPairs = []; - context.kvPairsObj = []; // New insert if (!performRMW) context.PopulatePrimary(ref context.kvPairs, keyLength, kvpairCount, primaryIndex: primaryIndex); @@ -57,7 +55,7 @@ void PopulatePrimary(int primaryIndex, bool disableObjects, bool performRMW) context.PopulatePrimaryRMW(ref context.kvPairs, keyLength, kvpairCount, primaryIndex: 0, addCount); if (!disableObjects) - context.PopulatePrimaryWithObjects(ref context.kvPairsObj, keyLength, kvpairCount, primaryIndex: 0); + context.PopulatePrimaryWithObjects(ref context.kvPairsObj, keyLength, kvpairCount, primaryIndex: primaryIndex); } void Validate(int primaryIndex, int replicaIndex, bool disableObjects) @@ -81,6 +79,18 @@ void ResetAndReAttach(int replicaIndex, int primaryIndex, bool soft) } } + void Failover(int replicaIndex, string option = null) + { + _ = context.clusterTestUtils.ClusterFailover(replicaIndex, option, logger: context.logger); + var role = context.clusterTestUtils.RoleCommand(replicaIndex, logger: context.logger); + while (!role.Value.Equals("master")) + { + ClusterTestUtils.BackOff(cancellationToken: context.cts.Token); + role = context.clusterTestUtils.RoleCommand(replicaIndex, logger: context.logger); + } + + } + /// /// Attach empty replica after primary has been populated with some data /// @@ -88,7 +98,7 @@ void ResetAndReAttach(int replicaIndex, int primaryIndex, bool soft) /// [Test, Order(1)] [Category("REPLICATION")] - public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Values] bool performRMW) + public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Values] bool performRMW, [Values] bool prePopulate) { var nodes_count = 2; var primaryIndex = 0; @@ -106,11 +116,14 @@ public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Value context.clusterTestUtils.WaitUntilNodeIsKnown(primaryIndex, replicaIndex, logger: context.logger); // Populate Primary - PopulatePrimary(primaryIndex, disableObjects, performRMW); + if (prePopulate) PopulatePrimary(primaryIndex, disableObjects, performRMW); // Attach sync session _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaIndex, primaryNodeIndex: primaryIndex, logger: context.logger); + // Populate Primary + if (!prePopulate) PopulatePrimary(primaryIndex, disableObjects, performRMW); + // Wait for replica to catch up context.clusterTestUtils.WaitForReplicaAofSync(primaryIndex, replicaIndex, logger: context.logger); @@ -118,7 +131,6 @@ public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Value Validate(primaryIndex, replicaIndex, disableObjects); } - /// /// Re-attach replica on disconnect after it has received some amount of data. /// The replica should replay only the portion it missed without doing a full sync @@ -201,6 +213,7 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, // Ensure node everybody knowns everybody context.clusterTestUtils.WaitUntilNodeIsKnown(replicaOneIndex, primaryIndex, logger: context.logger); context.clusterTestUtils.WaitUntilNodeIsKnown(replicaTwoIndex, primaryIndex, logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaTwoIndex, replicaOneIndex, logger: context.logger); // Populate Primary PopulatePrimary(primaryIndex, disableObjects, performRMW); @@ -250,5 +263,109 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); } + + [Test, Order(4)] + [Category("REPLICATION")] + public void ClusterDisklessSyncParallelAttach([Values] bool disableObjects, [Values] bool performRMW) + { + var nodes_count = 4; + var primaryIndex = 0; + var replicaOneIndex = 1; + var replicaTwoIndex = 2; + var replicaThreeIndex = 3; + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateConnection(useTLS: useTLS); + + // Setup primary and introduce it to future replica + _ = context.clusterTestUtils.AddDelSlotsRange(primaryIndex, [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(primaryIndex, primaryIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaOneIndex, replicaOneIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaTwoIndex, replicaTwoIndex + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(replicaThreeIndex, replicaThreeIndex + 1, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaOneIndex, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaTwoIndex, logger: context.logger); + context.clusterTestUtils.Meet(primaryIndex, replicaThreeIndex, logger: context.logger); + + // Ensure node everybody knowns everybody + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaOneIndex, primaryIndex, logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaTwoIndex, primaryIndex, logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(replicaThreeIndex, primaryIndex, logger: context.logger); + + // Populate Primary + for (var i = 0; i < 5; i++) + PopulatePrimary(primaryIndex, disableObjects, performRMW); + + // Attach all replicas + for (var replica = 1; replica < nodes_count; replica++) + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replica, primaryNodeIndex: primaryIndex, logger: context.logger); + + // Validate all replicas + for (var replica = 1; replica < nodes_count; replica++) + Validate(primaryIndex, replica, disableObjects); + } + + [Test, Order(5)] + [Category("REPLICATION")] + public void ClusterDisklessSyncFailover([Values] bool disableObjects, [Values] bool performRMW) + { + var nodes_count = 3; + var primary = 0; + var replicaOne = 1; + var replicaTwo = 2; + var populateIter = 1; + + int[] nOffsets = [primary, replicaOne, replicaTwo]; + + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateConnection(useTLS: useTLS); + + // Setup primary and introduce it to future replica + _ = context.clusterTestUtils.AddDelSlotsRange(nOffsets[primary], [(0, 16383)], addslot: true, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(nOffsets[primary], nOffsets[primary] + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(nOffsets[replicaOne], nOffsets[replicaOne] + 1, logger: context.logger); + context.clusterTestUtils.SetConfigEpoch(nOffsets[replicaTwo], nOffsets[replicaTwo] + 1, logger: context.logger); + context.clusterTestUtils.Meet(nOffsets[primary], nOffsets[replicaOne], logger: context.logger); + context.clusterTestUtils.Meet(nOffsets[primary], nOffsets[replicaTwo], logger: context.logger); + + context.clusterTestUtils.WaitUntilNodeIsKnown(nOffsets[primary], nOffsets[replicaOne], logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(nOffsets[replicaOne], nOffsets[replicaTwo], logger: context.logger); + context.clusterTestUtils.WaitUntilNodeIsKnown(nOffsets[replicaTwo], nOffsets[primary], logger: context.logger); + + // Populate Primary + for (var i = 0; i < populateIter; i++) + PopulatePrimary(nOffsets[primary], disableObjects, performRMW); + + // Attach replicas + for (var replica = 1; replica < nodes_count; replica++) + _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replica, primaryNodeIndex: nOffsets[primary], logger: context.logger); + + // Wait for replica to catch up + for (var replica = 1; replica < nodes_count; replica++) + context.clusterTestUtils.WaitForReplicaAofSync(nOffsets[primary], nOffsets[replica], logger: context.logger); + + // Validate all replicas + for (var replica = 1; replica < nodes_count; replica++) + Validate(nOffsets[primary], nOffsets[replica], disableObjects); + + // Perform failover and promote replica one + Failover(nOffsets[replicaOne]); + (nOffsets[replicaOne], nOffsets[primary]) = (nOffsets[primary], nOffsets[replicaOne]); + + // Wait for replica to catch up + for (var replica = 1; replica < nodes_count; replica++) + context.clusterTestUtils.WaitForReplicaAofSync(nOffsets[primary], nOffsets[replica], logger: context.logger); + + // Populate Primary + for (var i = 0; i < populateIter; i++) + PopulatePrimary(nOffsets[primary], disableObjects, performRMW); + + // Wait for replica to catch up + for (var replica = 1; replica < nodes_count; replica++) + context.clusterTestUtils.WaitForReplicaAofSync(nOffsets[primary], nOffsets[replica], logger: context.logger); + + // Validate all replicas + for (var replica = 1; replica < nodes_count; replica++) + Validate(nOffsets[primary], nOffsets[replica], disableObjects); + } } } \ No newline at end of file From 2d91745a9015122eba507641a709ec9cda365edc Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 4 Feb 2025 14:27:01 -0800 Subject: [PATCH 21/32] fix formatting and conversion to IPEndpoint --- .../Server/Replication/PrimaryOps/AofTaskStore.cs | 3 +-- .../PrimaryOps/DisklessReplication/SyncStatus.cs | 2 +- .../Replication/ReplicaOps/ReplicaDisklessSync.cs | 10 +++++----- libs/host/Configuration/Options.cs | 1 + test/Garnet.test.cluster/ClusterManagementTests.cs | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs index dbd66816e5..aa2494104e 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs @@ -239,8 +239,7 @@ public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, lon current.LocalNodeId, replicaNodeId, new GarnetClientSession( - address, - port, + new IPEndPoint(IPAddress.Parse(address), port), clusterProvider.replicationManager.GetAofSyncNetworkBufferSettings, clusterProvider.replicationManager.GetNetworkPool, tlsOptions: clusterProvider.serverOptions.TlsOptions?.TlsClientOptions, diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs index 6dd8eaf131..b06cfc8644 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/SyncStatus.cs @@ -22,4 +22,4 @@ struct SyncStatusInfo public SyncStatus syncStatus; public string error; } -} +} \ No newline at end of file diff --git a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs index 77cde55e0b..8a3a56ed66 100644 --- a/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs +++ b/libs/cluster/Server/Replication/ReplicaOps/ReplicaDisklessSync.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Net; using System.Text; using System.Threading.Tasks; using Garnet.client; @@ -87,10 +88,9 @@ async Task TryBeginReplicaSync() } gcs = new( - address, - port, - clusterProvider.replicationManager.GetIRSNetworkBufferSettings, - clusterProvider.replicationManager.GetNetworkPool, + new IPEndPoint(IPAddress.Parse(address), port), + networkBufferSettings: clusterProvider.replicationManager.GetIRSNetworkBufferSettings, + networkPool: clusterProvider.replicationManager.GetNetworkPool, tlsOptions: clusterProvider.serverOptions.TlsOptions?.TlsClientOptions, authUsername: clusterProvider.ClusterUsername, authPassword: clusterProvider.ClusterPassword); @@ -167,4 +167,4 @@ public long ReplicaRecoverDiskless(SyncMetadata primarySyncMetadata) } } } -} +} \ No newline at end of file diff --git a/libs/host/Configuration/Options.cs b/libs/host/Configuration/Options.cs index 45045255a5..a492ec3c13 100644 --- a/libs/host/Configuration/Options.cs +++ b/libs/host/Configuration/Options.cs @@ -9,6 +9,7 @@ using System.Net; using System.Net.Sockets; using System.Reflection; +using System.Runtime.InteropServices; using System.Security.Cryptography.X509Certificates; using CommandLine; using Garnet.server; diff --git a/test/Garnet.test.cluster/ClusterManagementTests.cs b/test/Garnet.test.cluster/ClusterManagementTests.cs index f60aa1a5ed..6033e58696 100644 --- a/test/Garnet.test.cluster/ClusterManagementTests.cs +++ b/test/Garnet.test.cluster/ClusterManagementTests.cs @@ -734,14 +734,14 @@ public void ClusterNodeCommand() var lines = result.ToString().Split("\n", StringSplitOptions.RemoveEmptyEntries); ClassicAssert.AreEqual(node_count, lines.Length); - var primaries = shardInfo[0].nodes.Where(x => x.role == Role.PRIMARY).Select(w => w.nodeid).ToArray(); + var primaries = shardInfo[0].nodes.Where(x => x.role == NodeRole.PRIMARY).Select(w => w.nodeid).ToArray(); foreach (var line in lines) { var fields = line.Split(' '); ClassicAssert.IsTrue(shardInfo[0].nodes.Any(e => e.nodeid == fields[0])); var node = shardInfo[0].nodes.Single(e => e.nodeid == fields[0]); - if (node.role == Role.PRIMARY) + if (node.role == NodeRole.PRIMARY) { ClassicAssert.GreaterOrEqual(fields.Length, 8); ClassicAssert.IsTrue(fields[1].StartsWith("127.0.0.1")); From 456e455b8e76dc11d82a5f83fb4ec58121aa332b Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 4 Feb 2025 16:50:26 -0800 Subject: [PATCH 22/32] fix RepCommandsTests --- test/Garnet.test/Resp/ACL/RespCommandTests.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/Garnet.test/Resp/ACL/RespCommandTests.cs b/test/Garnet.test/Resp/ACL/RespCommandTests.cs index 5b59c2bca8..216a80cfac 100644 --- a/test/Garnet.test/Resp/ACL/RespCommandTests.cs +++ b/test/Garnet.test/Resp/ACL/RespCommandTests.cs @@ -93,6 +93,7 @@ public void AllCommandsCovered() { // Exclude things like ACL, CLIENT, CLUSTER which are "commands" but only their sub commands can be run IEnumerable subCommands = allInfo.Where(static x => x.Value.SubCommands != null).SelectMany(static x => x.Value.SubCommands).Select(static x => x.Name); + var x = advertisedCommands.Except(withOnlySubCommands).Union(subCommands); IEnumerable deSubCommanded = advertisedCommands.Except(withOnlySubCommands).Union(subCommands).Select(static x => x.Replace("|", "").Replace("_", "").Replace("-", "")); IEnumerable notCovered = deSubCommanded.Except(covered, StringComparer.OrdinalIgnoreCase).Except(notCoveredByACLs, StringComparer.OrdinalIgnoreCase); @@ -1006,6 +1007,7 @@ static async Task DoClusterAppendLogAsync(GarnetClient client) } } + [Test] public async Task ClusterAttachSyncACLsAsync() { // All cluster command "success" is a thrown exception, because clustering is disabled From a0807a8315aa4a713faf50200515b8150625220e Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 4 Feb 2025 18:31:19 -0800 Subject: [PATCH 23/32] dispose aofSyncTask if failed to add to AofSyncTaskStore --- .../DisklessReplication/ReplicaSyncSession.cs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index d550bda964..a6246bf07d 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -248,9 +248,9 @@ bool ShouldStreamDisklessCheckpoint() /// public async Task BeginAofSync() { + var aofSyncTask = AofSyncTask; try { - var aofSyncTask = AofSyncTask; var mmr = clusterProvider.serverOptions.MainMemoryReplication; var aofNull = clusterProvider.serverOptions.UseAofNullDevice; @@ -280,11 +280,18 @@ public async Task BeginAofSync() // We have already added the iterator for the covered address above but replica might request an address // that is ahead of the covered address so we should start streaming from that address in order not to // introduce duplicate insertions. - if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaSyncMetadata.originNodeId, syncFromAofAddress, out var aofSyncTaskInfo)) + if (!clusterProvider.replicationManager.TryAddReplicationTask(replicaSyncMetadata.originNodeId, syncFromAofAddress, out aofSyncTask)) throw new GarnetException("Failed trying to try update replication task"); - if (!clusterProvider.replicationManager.TryConnectToReplica(replicaSyncMetadata.originNodeId, syncFromAofAddress, aofSyncTaskInfo, out _)) + if (!clusterProvider.replicationManager.TryConnectToReplica(replicaSyncMetadata.originNodeId, syncFromAofAddress, aofSyncTask, out _)) throw new GarnetException("Failed connecting to replica for aofSync"); } + catch (Exception ex) + { + logger?.LogError(ex, "{method}", $"{nameof(ReplicaSyncSession.BeginAofSync)}"); + SetStatus(SyncStatus.FAILED, "Failed to begin AOF sync"); + aofSyncTask?.Dispose(); + Dispose(); + } finally { AofSyncTask = null; From 8575064ffa1040267d940d6a2bcbba524ada0798 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Tue, 4 Feb 2025 18:56:50 -0800 Subject: [PATCH 24/32] overload dispose ReplicaSyncSession --- .../PrimaryOps/DisklessReplication/ReplicaSyncSession.cs | 3 +-- .../PrimaryOps/DisklessReplication/ReplicationSyncManager.cs | 2 +- .../Server/Replication/PrimaryOps/ReplicaSyncSession.cs | 5 ++++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index a6246bf07d..3b2d223cae 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -289,8 +289,7 @@ public async Task BeginAofSync() { logger?.LogError(ex, "{method}", $"{nameof(ReplicaSyncSession.BeginAofSync)}"); SetStatus(SyncStatus.FAILED, "Failed to begin AOF sync"); - aofSyncTask?.Dispose(); - Dispose(); + Dispose(true); } finally { diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index a7887ac523..4fde6c95a8 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -121,7 +121,7 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn if (replicaSyncSession.Failed) { replicaSyncSession.LogError(); - replicaSyncSession.Dispose(); + replicaSyncSession.Dispose(true); return replicaSyncSession.GetSyncStatusInfo; } diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index f6e2729086..896ec7b484 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -48,8 +48,11 @@ internal sealed partial class ReplicaSyncSession( public string errorMsg = default; - public void Dispose() + public void Dispose() => Dispose(false); + + public void Dispose(bool disposeAofSyncTask) { + if (disposeAofSyncTask) AofSyncTask?.Dispose(); AofSyncTask = null; cts.Cancel(); cts.Dispose(); From 7195c2d55205c76726515940d798a58f3fe10f20 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 5 Feb 2025 09:33:27 -0800 Subject: [PATCH 25/32] explicitly dispose gcs used for full sync at replicaSyncSession sync --- .../Replication/PrimaryOps/AofSyncTaskInfo.cs | 21 +------------------ .../DisklessReplication/ReplicaSyncSession.cs | 1 + .../Garnet.test.cluster/ClusterTestContext.cs | 5 +++-- test/Garnet.test.cluster/ClusterTestUtils.cs | 2 +- 4 files changed, 6 insertions(+), 23 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/AofSyncTaskInfo.cs b/libs/cluster/Server/Replication/PrimaryOps/AofSyncTaskInfo.cs index 7065ece0e4..611fdc5077 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/AofSyncTaskInfo.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/AofSyncTaskInfo.cs @@ -5,7 +5,6 @@ using System.Threading; using System.Threading.Tasks; using Garnet.client; -using Garnet.common; using Microsoft.Extensions.Logging; using Tsavorite.core; @@ -24,11 +23,6 @@ internal sealed class AofSyncTaskInfo : IBulkLogEntryConsumer, IDisposable readonly long startAddress; public long previousAddress; - /// - /// Used to mark if syncing is in progress - /// - SingleWriterMultiReaderLock aofSyncInProgress; - /// /// Check if client connection is healthy /// @@ -69,11 +63,6 @@ public void Dispose() // Finally, dispose the cts cts?.Dispose(); - - // Dispose only if AOF sync has not started - // otherwise sync task will dispose the client - if (aofSyncInProgress.TryWriteLock()) - garnetClient?.Dispose(); } public unsafe void Consume(byte* payloadPtr, int payloadLength, long currentAddress, long nextAddress, bool isProtected) @@ -108,16 +97,8 @@ public async Task ReplicaSyncTask() { logger?.LogInformation("Starting ReplicationManager.ReplicaSyncTask for remote node {remoteNodeId} starting from address {address}", remoteNodeId, startAddress); - var failedToStart = false; try { - if (!aofSyncInProgress.TryWriteLock()) - { - logger?.LogWarning("{method} AOF sync for {remoteNodeId} failed to start", nameof(ReplicaSyncTask), remoteNodeId); - failedToStart = true; - return; - } - if (!IsConnected) garnetClient.Connect(); iter = clusterProvider.storeWrapper.appendOnlyFile.ScanSingle(startAddress, long.MaxValue, scanUncommitted: true, recover: false, logger: logger); @@ -134,7 +115,7 @@ public async Task ReplicaSyncTask() } finally { - if (!failedToStart) garnetClient.Dispose(); + garnetClient.Dispose(); var (address, port) = clusterProvider.clusterManager.CurrentConfig.GetWorkerAddressFromNodeId(remoteNodeId); logger?.LogWarning("AofSync task terminated; client disposed {remoteNodeId} {address} {port} {currentAddress}", remoteNodeId, address, port, previousAddress); diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index 3b2d223cae..61e8e7ee85 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -293,6 +293,7 @@ public async Task BeginAofSync() } finally { + AofSyncTask.garnetClient.Dispose(); AofSyncTask = null; } } diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index ef64cbf02f..378ea79204 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -65,9 +65,10 @@ public void TearDown() waiter?.Dispose(); clusterTestUtils?.Dispose(); loggerFactory?.Dispose(); - if (!Task.Run(() => DisposeCluster()).Wait(TimeSpan.FromSeconds(60))) + var timeoutSeconds = 5; + if (!Task.Run(() => DisposeCluster()).Wait(TimeSpan.FromSeconds(timeoutSeconds))) logger?.LogError("Timed out waiting for DisposeCluster"); - if (!Task.Run(() => TestUtils.DeleteDirectory(TestFolder, true)).Wait(TimeSpan.FromSeconds(60))) + if (!Task.Run(() => TestUtils.DeleteDirectory(TestFolder, true)).Wait(TimeSpan.FromSeconds(timeoutSeconds))) logger?.LogError("Timed out waiting for DisposeCluster"); } diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index 93b8380cf0..20410d7177 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -2794,7 +2794,7 @@ public void WaitForReplicaAofSync(int primaryIndex, int secondaryIndex, ILogger var secondaryReplicationOffset1 = GetReplicationOffset(secondaryIndex, logger); if (primaryReplicationOffset == secondaryReplicationOffset1) break; - BackOff(cancellationToken: context.cts.Token); + BackOff(cancellationToken: context.cts.Token, msg: $"[{endpoints[primaryIndex]}]{primaryReplicationOffset} != [{endpoints[secondaryIndex]}]{secondaryReplicationOffset1}"); } logger?.LogInformation("Replication offset for primary {primaryIndex} and secondary {secondaryIndex} is {primaryReplicationOffset}", primaryIndex, secondaryIndex, primaryReplicationOffset); } From bdee03c4f56c7e1aa45da09cf71150e353b59da0 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 5 Feb 2025 11:07:11 -0800 Subject: [PATCH 26/32] dispose gcs once on return --- .../PrimaryOps/DisklessReplication/ReplicaSyncSession.cs | 7 +------ .../DisklessReplication/ReplicationSyncManager.cs | 2 -- .../Server/Replication/PrimaryOps/ReplicaSyncSession.cs | 6 ++---- 3 files changed, 3 insertions(+), 12 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index 61e8e7ee85..6236874946 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -289,12 +289,7 @@ public async Task BeginAofSync() { logger?.LogError(ex, "{method}", $"{nameof(ReplicaSyncSession.BeginAofSync)}"); SetStatus(SyncStatus.FAILED, "Failed to begin AOF sync"); - Dispose(true); - } - finally - { - AofSyncTask.garnetClient.Dispose(); - AofSyncTask = null; + AofSyncTask?.Dispose(); } } } diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 4fde6c95a8..1d1e0ba916 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -121,13 +121,11 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn if (replicaSyncSession.Failed) { replicaSyncSession.LogError(); - replicaSyncSession.Dispose(true); return replicaSyncSession.GetSyncStatusInfo; } // Start AOF sync background task for this replica await replicaSyncSession.BeginAofSync(); - return replicaSyncSession.GetSyncStatusInfo; } finally diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs index 896ec7b484..1b317c1c6d 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSession.cs @@ -48,11 +48,9 @@ internal sealed partial class ReplicaSyncSession( public string errorMsg = default; - public void Dispose() => Dispose(false); - - public void Dispose(bool disposeAofSyncTask) + public void Dispose() { - if (disposeAofSyncTask) AofSyncTask?.Dispose(); + AofSyncTask?.garnetClient?.Dispose(); AofSyncTask = null; cts.Cancel(); cts.Dispose(); From 5df44b74ecaac877a05a0397fe9e6de1ea12aaa1 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Wed, 5 Feb 2025 21:12:43 -0800 Subject: [PATCH 27/32] code cleanup --- .../DisklessReplication/ReplicaSyncSession.cs | 2 + .../ReplicationSyncManager.cs | 54 +++++---- .../PrimaryOps/ReplicaSyncSessionTaskStore.cs | 103 +++++++++++++----- .../Server/Replication/ReplicationManager.cs | 1 - 4 files changed, 106 insertions(+), 54 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index 6236874946..d5dfc9621d 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -27,6 +27,8 @@ internal sealed partial class ReplicaSyncSession public bool Failed => ssInfo.syncStatus == SyncStatus.FAILED; + public bool InProgress => ssInfo.syncStatus == SyncStatus.INPROGRESS; + public SyncStatusInfo GetSyncStatusInfo => ssInfo; public long currentStoreVersion; diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 1d1e0ba916..c86f13d8f1 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -106,8 +106,8 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn // Started syncing replicaSyncSession.SetStatus(SyncStatus.INPROGRESS); - var isLeader = GetSessionStore.GetSessions()[0] == replicaSyncSession; // Only one thread will acquire this lock + var isLeader = GetSessionStore.IsFirst(replicaSyncSession); if (isLeader) { // Launch a background task to sync the attached replicas using streaming snapshot @@ -141,14 +141,22 @@ async Task StreamingSnapshotSync() var disklessRepl = ClusterProvider.serverOptions.ReplicaDisklessSync; var disableObjects = ClusterProvider.serverOptions.DisableObjects; - NumSessions = GetSessionStore.GetNumSessions(); - Sessions = GetSessionStore.GetSessions(); - try { // Lock to avoid the addition of new replica sync sessions while sync is in progress syncInProgress.WriteLock(); + // Get sync session info + NumSessions = GetSessionStore.GetNumSessions(); + Sessions = GetSessionStore.GetSessions(); + + // Wait for all replicas to reach initializing state + for (var i = 0; i < NumSessions; i++) + { + while (!Sessions[i].InProgress) + await Task.Yield(); + } + // Take lock to ensure no other task will be taking a checkpoint while (!ClusterProvider.storeWrapper.TryPauseCheckpoints()) await Task.Yield(); @@ -159,25 +167,6 @@ async Task StreamingSnapshotSync() // Stream checkpoint to replicas await TakeStreamingCheckpoint(); - // Stream Diskless - async Task TakeStreamingCheckpoint() - { - // Main snapshot iterator manager - var manager = new SnapshotIteratorManager(this, cts.Token, logger); - - // Iterate through main store - var mainStoreResult = await ClusterProvider.storeWrapper.store. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); - - if (!ClusterProvider.serverOptions.DisableObjects) - { - // Iterate through object store - var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); - } - - ClusterProvider.replicationManager.SafeTruncateAof(manager.CheckpointCoveredAddress); - } - // Notify sync session of success success for (var i = 0; i < NumSessions; i++) Sessions[i]?.SetStatus(SyncStatus.SUCCESS); @@ -256,6 +245,25 @@ async Task PrepareForSync() #endregion } } + + // Stream Diskless + async Task TakeStreamingCheckpoint() + { + // Main snapshot iterator manager + var manager = new SnapshotIteratorManager(this, cts.Token, logger); + + // Iterate through main store + var mainStoreResult = await ClusterProvider.storeWrapper.store. + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + + if (!ClusterProvider.serverOptions.DisableObjects) + { + // Iterate through object store + var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + } + + ClusterProvider.replicationManager.SafeTruncateAof(manager.CheckpointCoveredAddress); + } } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs index fb6d288b0e..dc203f4f21 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/ReplicaSyncSessionTaskStore.cs @@ -16,9 +16,12 @@ internal sealed class ReplicaSyncSessionTaskStore int numSessions; SingleWriterMultiReaderLock _lock; readonly ILogger logger; - private bool _disposed; + public ReplicaSyncSession[] GetSessions() => sessions; + + public int GetNumSessions() => numSessions; + public ReplicaSyncSessionTaskStore(StoreWrapper storeWrapper, ClusterProvider clusterProvider, ILogger logger = null) { this.storeWrapper = storeWrapper; @@ -28,13 +31,16 @@ public ReplicaSyncSessionTaskStore(StoreWrapper storeWrapper, ClusterProvider cl this.logger = logger; } + /// + /// Dispose this Replica Sync Session Task Store + /// public void Dispose() { try { _lock.WriteLock(); _disposed = true; - for (int i = 0; i < numSessions; i++) + for (var i = 0; i < numSessions; i++) { var s = sessions[i]; s.Dispose(); @@ -48,6 +54,31 @@ public void Dispose() } } + /// + /// Check if the session is the first in the array + /// NOTE: used for leader task spawn in AttachSync + /// + /// + /// + public bool IsFirst(ReplicaSyncSession session) + { + try + { + _lock.ReadLock(); + if (_disposed) return false; + return numSessions > 0 && sessions[0] == session; + } + finally + { + _lock.ReadUnlock(); + } + } + + /// + /// Add a new replica sync session + /// + /// + /// public bool TryAddReplicaSyncSession(ReplicaSyncSession session) { try @@ -78,6 +109,15 @@ public bool TryAddReplicaSyncSession(ReplicaSyncSession session) } } + /// + /// Add a new replica sync session + /// + /// + /// + /// + /// + /// + /// public bool TryAddReplicaSyncSession(string replicaNodeId, string replicaAssignedPrimaryId, CheckpointEntry replicaCheckpointEntry, long replicaAofBeginAddress, long replicaAofTailAddress) { var retSession = new ReplicaSyncSession(storeWrapper, clusterProvider, replicaSyncMetadata: null, timeout: default, token: default, replicaNodeId, replicaAssignedPrimaryId, replicaCheckpointEntry, replicaAofBeginAddress, replicaAofTailAddress, logger); @@ -114,16 +154,6 @@ public bool TryAddReplicaSyncSession(string replicaNodeId, string replicaAssigne } } - private void GrowSessionArray() - { - if (numSessions == sessions.Length) - { - var _sessions = new ReplicaSyncSession[sessions.Length << 1]; - Array.Copy(sessions, _sessions, sessions.Length); - sessions = _sessions; - } - } - public bool TryRemove(string remoteNodeId) { try @@ -156,19 +186,12 @@ public bool TryRemove(string remoteNodeId) } } - private void ShrinkSessionArray() - { - //Shrink the array if it got too big but avoid often shrinking/growing - if (numSessions > 0 && (numSessions << 2) < sessions.Length) - { - var oldSessions = sessions; - var _sessions = new ReplicaSyncSession[sessions.Length >> 1]; - Array.Copy(sessions, _sessions, sessions.Length >> 2); - sessions = _sessions; - Array.Clear(oldSessions); - } - } - + /// + /// Get session associated with replica sync session + /// + /// + /// + /// public bool TryGetSession(string remoteNodeId, out ReplicaSyncSession session) { session = null; @@ -177,7 +200,7 @@ public bool TryGetSession(string remoteNodeId, out ReplicaSyncSession session) _lock.ReadLock(); if (_disposed) return false; - for (int i = 0; i < numSessions; i++) + for (var i = 0; i < numSessions; i++) { session = sessions[i]; if (session.replicaNodeId == remoteNodeId) @@ -191,12 +214,9 @@ public bool TryGetSession(string remoteNodeId, out ReplicaSyncSession session) } } - public ReplicaSyncSession[] GetSessions() => sessions; - - public int GetNumSessions() => numSessions; /// - /// Clear references to entries + /// Clear references to task store /// /// public void Clear() @@ -217,5 +237,28 @@ public void Clear() _lock.WriteUnlock(); } } + + private void GrowSessionArray() + { + if (numSessions == sessions.Length) + { + var _sessions = new ReplicaSyncSession[sessions.Length << 1]; + Array.Copy(sessions, _sessions, sessions.Length); + sessions = _sessions; + } + } + + private void ShrinkSessionArray() + { + //Shrink the array if it got too big but avoid often shrinking/growing + if (numSessions > 0 && (numSessions << 2) < sessions.Length) + { + var oldSessions = sessions; + var _sessions = new ReplicaSyncSession[sessions.Length >> 1]; + Array.Copy(sessions, _sessions, sessions.Length >> 2); + sessions = _sessions; + Array.Clear(oldSessions); + } + } } } \ No newline at end of file diff --git a/libs/cluster/Server/Replication/ReplicationManager.cs b/libs/cluster/Server/Replication/ReplicationManager.cs index e70dd3614d..cde395995e 100644 --- a/libs/cluster/Server/Replication/ReplicationManager.cs +++ b/libs/cluster/Server/Replication/ReplicationManager.cs @@ -21,7 +21,6 @@ internal sealed partial class ReplicationManager : IDisposable readonly CheckpointStore checkpointStore; readonly ReplicationSyncManager replicationSyncManager; - readonly CancellationTokenSource ctsRepManager = new(); readonly int pageSizeBits; From e2be1b78058763fa68fe4ab95a68f13edcda5e0e Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 6 Feb 2025 08:18:29 -0800 Subject: [PATCH 28/32] update tests to provide more context logging --- test/Garnet.test.cluster/ClusterTestContext.cs | 4 ++-- test/Garnet.test.cluster/ClusterTestUtils.cs | 10 +++++++--- .../ClusterReplicationDisklessSyncTests.cs | 4 +++- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index 378ea79204..c6b563d5cb 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -41,7 +41,7 @@ public class ClusterTestContext public CancellationTokenSource cts; - public void Setup(Dictionary monitorTests, int testTimeoutSeconds = 60) + public void Setup(Dictionary monitorTests, int testTimeoutSeconds = 30) { cts = new CancellationTokenSource(TimeSpan.FromSeconds(testTimeoutSeconds)); @@ -468,7 +468,7 @@ public void ValidateKVCollectionAgainstReplica( while (responseState != ResponseState.OK || retVal == null || (value != int.Parse(retVal))) { retVal = clusterTestUtils.GetKey(replicaIndex, keyBytes, out _, out _, out responseState, logger: logger); - ClusterTestUtils.BackOff(cancellationToken: cts.Token); + ClusterTestUtils.BackOff(cancellationToken: cts.Token, msg: $"{clusterTestUtils.GetEndPoint(primaryIndex)} > {clusterTestUtils.GetEndPoint(replicaIndex)}"); } ClassicAssert.AreEqual(ResponseState.OK, responseState); ClassicAssert.AreEqual(value, int.Parse(retVal), $"replOffset > p:{clusterTestUtils.GetReplicationOffset(primaryIndex, logger: logger)}, s[{replicaIndex}]:{clusterTestUtils.GetReplicationOffset(replicaIndex)}"); diff --git a/test/Garnet.test.cluster/ClusterTestUtils.cs b/test/Garnet.test.cluster/ClusterTestUtils.cs index 20410d7177..d3baecc4a8 100644 --- a/test/Garnet.test.cluster/ClusterTestUtils.cs +++ b/test/Garnet.test.cluster/ClusterTestUtils.cs @@ -2788,15 +2788,19 @@ public string GetInfo(IPEndPoint endPoint, string section, string segment, ILogg public void WaitForReplicaAofSync(int primaryIndex, int secondaryIndex, ILogger logger = null) { long primaryReplicationOffset; + long secondaryReplicationOffset1; while (true) { primaryReplicationOffset = GetReplicationOffset(primaryIndex, logger); - var secondaryReplicationOffset1 = GetReplicationOffset(secondaryIndex, logger); + secondaryReplicationOffset1 = GetReplicationOffset(secondaryIndex, logger); if (primaryReplicationOffset == secondaryReplicationOffset1) break; - BackOff(cancellationToken: context.cts.Token, msg: $"[{endpoints[primaryIndex]}]{primaryReplicationOffset} != [{endpoints[secondaryIndex]}]{secondaryReplicationOffset1}"); + + var primaryMainStoreVersion = context.clusterTestUtils.GetStoreCurrentVersion(primaryIndex, isMainStore: true, logger); + var replicaMainStoreVersion = context.clusterTestUtils.GetStoreCurrentVersion(secondaryIndex, isMainStore: true, logger); + BackOff(cancellationToken: context.cts.Token, msg: $"[{endpoints[primaryIndex]}]: {primaryMainStoreVersion},{primaryReplicationOffset} != [{endpoints[secondaryIndex]}]: {replicaMainStoreVersion},{secondaryReplicationOffset1}"); } - logger?.LogInformation("Replication offset for primary {primaryIndex} and secondary {secondaryIndex} is {primaryReplicationOffset}", primaryIndex, secondaryIndex, primaryReplicationOffset); + logger?.LogInformation("[{primaryEndpoint}]{primaryReplicationOffset} ?? [{endpoints[secondaryEndpoint}]{secondaryReplicationOffset1}", endpoints[primaryIndex], primaryReplicationOffset, endpoints[secondaryIndex], secondaryReplicationOffset1); } public void WaitForConnectedReplicaCount(int primaryIndex, long minCount, ILogger logger = null) diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index ca58f2d4a8..8b7e41ca2a 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -26,7 +26,9 @@ public class ClusterReplicationDisklessSyncTests protected bool useTLS = false; protected bool asyncReplay = false; - public Dictionary monitorTests = []; + public Dictionary monitorTests = new(){ + { "ClusterDisklessSyncFailover", LogLevel.Trace } + }; [SetUp] public virtual void Setup() From 0a366fd8643e061d0c72bda03b2b90b322b9e1fb Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 6 Feb 2025 10:52:05 -0800 Subject: [PATCH 29/32] add more comprehensive logging of syncMetadata --- .../DisklessReplication/ReplicaSyncSession.cs | 8 +- .../ReplicationSyncManager.cs | 6 +- .../Server/Replication/SyncMetadata.cs | 98 ++++++++++++++++++- 3 files changed, 107 insertions(+), 5 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs index d5dfc9621d..be7f00e73c 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicaSyncSession.cs @@ -43,13 +43,15 @@ internal sealed partial class ReplicaSyncSession /// /// LogError /// - public void LogError() + public void LogMetadata(LogLevel logLevel) { - logger?.LogError("{msg} > " + + logger?.Log(logLevel, + "{status} {msg} > " + "originNodeId: {originNodeId}, " + "currentPrimaryReplId: {currentPrimaryReplId}, " + "currentAofBeginAddress: {currentAofBeginAddress}, " + "currentAofTailAddress: {currentAofTailAddress}, ", + ssInfo.syncStatus, ssInfo.error, replicaSyncMetadata.originNodeId, replicaSyncMetadata.currentPrimaryReplId, @@ -279,6 +281,8 @@ public async Task BeginAofSync() return; } + logger?.LogSyncMetadata(LogLevel.Trace, "BeginAofSync", replicaSyncMetadata, recoverSyncMetadata); + // We have already added the iterator for the covered address above but replica might request an address // that is ahead of the covered address so we should start streaming from that address in order not to // introduce duplicate insertions. diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index c86f13d8f1..4ec7a9ed34 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -120,8 +120,10 @@ public async Task MainDisklessSync(ReplicaSyncSession replicaSyn // If session faulted return early if (replicaSyncSession.Failed) { - replicaSyncSession.LogError(); - return replicaSyncSession.GetSyncStatusInfo; + var status = replicaSyncSession.GetSyncStatusInfo; + var msg = $"{status.syncStatus}:{status.error}"; + logger?.LogSyncMetadata(LogLevel.Error, msg, replicaSyncSession.replicaSyncMetadata); + return status; } // Start AOF sync background task for this replica diff --git a/libs/cluster/Server/Replication/SyncMetadata.cs b/libs/cluster/Server/Replication/SyncMetadata.cs index 2b7e31d658..d208134645 100644 --- a/libs/cluster/Server/Replication/SyncMetadata.cs +++ b/libs/cluster/Server/Replication/SyncMetadata.cs @@ -3,10 +3,106 @@ using System.IO; using System.Text; +using Microsoft.Extensions.Logging; namespace Garnet.cluster { - sealed class SyncMetadata( + internal static class SyncMetadataLoggingExtensions + { + /// + /// Log sync metadata + /// + /// + /// + /// + /// + public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string msg, SyncMetadata syncMetadata) + { + log.Log(logLevel, + "\n" + + "[{msg}]\n" + + "fullSync:{fullSync}\n" + + "originNodeRole:{originNodeRole}\n" + + "originNodeId:{originNodeId}\n" + + "currentPrimaryReplId:{currentPrimaryReplId}\n" + + "currentStoreVersion:{currentStoreVersion}\n" + + "currentObjectStoreVersion:{currentObjectStoreVersion}\n" + + "currentAofBeginAddress:{currentAofBeginAddress}\n" + + "currentAofTailAddress:{currentAofTailAddress}\n" + + "currentReplicationOffset:{currentReplicationOffset}\n" + + "checkpointEntry:{checkpointEntry}", + msg, + syncMetadata.fullSync, + syncMetadata.originNodeRole, + syncMetadata.originNodeId, + syncMetadata.currentPrimaryReplId, + syncMetadata.currentStoreVersion, + syncMetadata.currentObjectStoreVersion, + syncMetadata.currentAofBeginAddress, + syncMetadata.currentAofTailAddress, + syncMetadata.currentReplicationOffset, + syncMetadata.checkpointEntry); + } + + /// + /// Log sync metadata + /// + /// + /// + /// + /// + /// + public static void LogSyncMetadata(this ILogger log, LogLevel logLevel, string msg, SyncMetadata origin, SyncMetadata local) + { + log.Log(logLevel, + "\n" + + "[{msg}]\n" + + "fullSync:{fullSync}\n" + + "originNodeRole:{originNodeRole}\n" + + "originNodeId:{originNodeId}\n" + + "currentPrimaryReplId:{currentPrimaryReplId}\n" + + "currentStoreVersion:{currentStoreVersion}\n" + + "currentObjectStoreVersion:{currentObjectStoreVersion}\n" + + "currentAofBeginAddress:{currentAofBeginAddress}\n" + + "currentAofTailAddress:{currentAofTailAddress}\n" + + "currentReplicationOffset:{currentReplicationOffset}\n" + + "checkpointEntry:{checkpointEntry}\n" + + "??\n" + + "recoverFullSync:{fullSync}\n" + + "recoverOriginNodeRole:{originNodeRole}\n" + + "recoverOriginNodeId:{originNodeId}\n" + + "recoverCurrentPrimaryReplId:{currentPrimaryReplId}\n" + + "recoverCurrentStoreVersion:{currentStoreVersion}\n" + + "recoverCurrentObjectStoreVersion:{currentObjectStoreVersion}\n" + + "recoverCurrentAofBeginAddress:{currentAofBeginAddress}\n" + + "recoverCurrentAofTailAddress:{currentAofTailAddress}\n" + + "recoverCurrentReplicationOffset:{currentReplicationOffset}\n" + + "recoverCheckpointEntry:{checkpointEntry}", + msg, + origin.fullSync, + origin.originNodeRole, + origin.originNodeId, + origin.currentPrimaryReplId, + origin.currentStoreVersion, + origin.currentObjectStoreVersion, + origin.currentAofBeginAddress, + origin.currentAofTailAddress, + origin.currentReplicationOffset, + origin.checkpointEntry, + local.fullSync, + local.originNodeRole, + local.originNodeId, + local.currentPrimaryReplId, + local.currentStoreVersion, + local.currentObjectStoreVersion, + local.currentAofBeginAddress, + local.currentAofTailAddress, + local.currentReplicationOffset, + local.checkpointEntry); + } + } + + internal sealed class SyncMetadata( bool fullSync, NodeRole originNodeRole, string originNodeId, From dcf91388b34fc008333611ff4fea007d4de219f6 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 6 Feb 2025 11:48:57 -0800 Subject: [PATCH 30/32] add timeout for streaming checkpoint --- .../DisklessReplication/ReplicationSyncManager.cs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs index 4ec7a9ed34..4736f83258 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSyncManager.cs @@ -256,12 +256,15 @@ async Task TakeStreamingCheckpoint() // Iterate through main store var mainStoreResult = await ClusterProvider.storeWrapper.store. - TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator); + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.mainStoreSnapshotIterator). + AsTask().WaitAsync(clusterTimeout, cts.Token); if (!ClusterProvider.serverOptions.DisableObjects) { // Iterate through object store - var objectStoreResult = await ClusterProvider.storeWrapper.objectStore.TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator); + var objectStoreResult = await ClusterProvider.storeWrapper.objectStore. + TakeFullCheckpointAsync(CheckpointType.StreamingSnapshot, streamingSnapshotIteratorFunctions: manager.objectStoreSnapshotIterator). + AsTask().WaitAsync(clusterTimeout, cts.Token); } ClusterProvider.replicationManager.SafeTruncateAof(manager.CheckpointCoveredAddress); From 393d8fa911aa40860c67188881a23a0199d6c545 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 6 Feb 2025 11:49:42 -0800 Subject: [PATCH 31/32] add clusterTimeout for diskless repl tests --- test/Garnet.test.cluster/ClusterTestContext.cs | 2 +- .../ClusterReplicationDisklessSyncTests.cs | 18 +++++++++++------- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/test/Garnet.test.cluster/ClusterTestContext.cs b/test/Garnet.test.cluster/ClusterTestContext.cs index c6b563d5cb..77b6b3101c 100644 --- a/test/Garnet.test.cluster/ClusterTestContext.cs +++ b/test/Garnet.test.cluster/ClusterTestContext.cs @@ -41,7 +41,7 @@ public class ClusterTestContext public CancellationTokenSource cts; - public void Setup(Dictionary monitorTests, int testTimeoutSeconds = 30) + public void Setup(Dictionary monitorTests, int testTimeoutSeconds = 60) { cts = new CancellationTokenSource(TimeSpan.FromSeconds(testTimeoutSeconds)); diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 8b7e41ca2a..5ab4c56c96 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Collections.Generic; using Microsoft.Extensions.Logging; using NUnit.Framework; @@ -26,15 +27,18 @@ public class ClusterReplicationDisklessSyncTests protected bool useTLS = false; protected bool asyncReplay = false; + int timeout = (int)TimeSpan.FromSeconds(15).TotalSeconds; + int testTimeout = (int)TimeSpan.FromSeconds(120).TotalSeconds; + public Dictionary monitorTests = new(){ - { "ClusterDisklessSyncFailover", LogLevel.Trace } + { "ClusterDisklessSyncFailover", LogLevel.Error } }; [SetUp] public virtual void Setup() { context = new ClusterTestContext(); - context.Setup(monitorTests); + context.Setup(monitorTests, testTimeoutSeconds: testTimeout); } [TearDown] @@ -105,7 +109,7 @@ public void ClusterEmptyReplicaDisklessSync([Values] bool disableObjects, [Value var nodes_count = 2; var primaryIndex = 0; var replicaIndex = 1; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -146,7 +150,7 @@ public void ClusterAofReplayDisklessSync([Values] bool disableObjects, [Values] var nodes_count = 2; var primaryIndex = 0; var replicaIndex = 1; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -201,7 +205,7 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, var primaryIndex = 0; var replicaOneIndex = 1; var replicaTwoIndex = 2; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -275,7 +279,7 @@ public void ClusterDisklessSyncParallelAttach([Values] bool disableObjects, [Val var replicaOneIndex = 1; var replicaTwoIndex = 2; var replicaThreeIndex = 3; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica @@ -318,7 +322,7 @@ public void ClusterDisklessSyncFailover([Values] bool disableObjects, [Values] b int[] nOffsets = [primary, replicaOne, replicaTwo]; - context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true); + context.CreateInstances(nodes_count, disableObjects: disableObjects, enableAOF: true, useTLS: useTLS, enableDisklessSync: true, timeout: timeout); context.CreateConnection(useTLS: useTLS); // Setup primary and introduce it to future replica From b8029611640040f141b90f8c114626d9a1ff85a4 Mon Sep 17 00:00:00 2001 From: Vasileios Zois Date: Thu, 6 Feb 2025 16:45:03 -0800 Subject: [PATCH 32/32] some more logging --- .../DisklessReplication/ReplicationSnapshotIterator.cs | 9 +++++++++ .../ClusterReplicationDisklessSyncTests.cs | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs index e9813fd8b6..83a572b68b 100644 --- a/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs +++ b/libs/cluster/Server/Replication/PrimaryOps/DisklessReplication/ReplicationSnapshotIterator.cs @@ -22,6 +22,8 @@ internal sealed unsafe class SnapshotIteratorManager readonly ReplicaSyncSession[] sessions; readonly int numSessions; + bool firstRead = false; + public long CheckpointCoveredAddress { get; private set; } public SnapshotIteratorManager(ReplicationSyncManager replicationSyncManager, CancellationToken cancellationToken, ILogger logger = null) @@ -79,6 +81,11 @@ void WaitForFlushAll() public bool Reader(ref SpanByte key, ref SpanByte value, RecordMetadata recordMetadata, long numberOfRecords) { + if (!firstRead) + { + logger?.LogTrace("First Read {key} {value}", key.ToString(), value.ToString()); + firstRead = true; + } var needToFlush = false; while (true) { @@ -162,6 +169,8 @@ public void OnStop(bool completed, long numberOfRecords, bool isMainStore, long logger?.LogTrace("{OnStop} {store} {numberOfRecords} {targetVersion}", nameof(OnStop), isMainStore ? "MAIN STORE" : "OBJECT STORE", numberOfRecords, targetVersion); + + firstRead = false; } } diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 5ab4c56c96..f050603957 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -31,7 +31,7 @@ public class ClusterReplicationDisklessSyncTests int testTimeout = (int)TimeSpan.FromSeconds(120).TotalSeconds; public Dictionary monitorTests = new(){ - { "ClusterDisklessSyncFailover", LogLevel.Error } + { "ClusterDisklessSyncFailover", LogLevel.Trace } }; [SetUp]