From e1db78435e2df01b84281778f54ea084afdabcf9 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 24 Feb 2025 13:22:59 -0800 Subject: [PATCH 01/49] Simpler state machine for checkpointing --- .../Index/Checkpointing/FoldOverSMTask.cs | 39 ++++ .../Index/Checkpointing/FullCheckpointSM.cs | 53 ++++++ .../Checkpointing/FullCheckpointSMTask.cs | 53 ++++++ .../Checkpointing/HybridLogCheckpointSM.cs | 43 +++++ .../HybridLogCheckpointSMTask.cs | 104 +++++++++++ .../core/Index/Checkpointing/IStateMachine.cs | 10 + .../Index/Checkpointing/IStateMachineTask.cs | 22 +++ .../Index/Checkpointing/IndexSnapshotSM.cs | 43 +++++ .../Checkpointing/IndexSnapshotSMTask.cs | 102 +++++++++++ .../Index/Checkpointing/StateMachineBase.cs | 43 +++++ .../Index/Checkpointing/StateMachineDriver.cs | 173 ++++++++++++++++++ .../Index/Checkpointing/VersionChangeSM.cs | 45 +++++ 12 files changed, 730 insertions(+) create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs new file mode 100644 index 0000000000..681a7c490c --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -0,0 +1,39 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// A FoldOver checkpoint persists a version by setting the read-only marker past the last entry of that + /// version on the log and waiting until it is flushed to disk. It is simple and fast, but can result + /// in garbage entries on the log, and a slower recovery of performance. + /// + internal sealed class FoldOverSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + public FoldOverSMTask(TsavoriteKV store) + : base(store) + { + } + + /// + public override void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + base.GlobalBeforeEnteringState(next, stateMachineDriver); + + if (next.Phase == Phase.PREPARE) + { + store._lastSnapshotCheckpoint.Dispose(); + } + + if (next.Phase == Phase.IN_PROGRESS) + base.GlobalBeforeEnteringState(next, stateMachineDriver); + + if (next.Phase != Phase.WAIT_FLUSH) return; + + _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); + store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs new file mode 100644 index 0000000000..0a1fef9350 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// The state machine orchestrates a full checkpoint + /// + internal sealed class FullCheckpointSM : HybridLogCheckpointSM + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + /// + /// Construct a new FullCheckpointStateMachine to use the given checkpoint backend (either fold-over or snapshot), + /// drawing boundary at targetVersion. + /// + /// A task that encapsulates the logic to persist the checkpoint + /// upper limit (inclusive) of the version included + public FullCheckpointSM(TsavoriteKV store, IStateMachineTask checkpointBackend, long targetVersion = -1) : base( + targetVersion, + new FullCheckpointSMTask(store), + new IndexSnapshotSMTask(store), + checkpointBackend) + { } + + /// + public override SystemState NextState(SystemState start, out bool barrier) + { + barrier = false; + var result = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.REST: + result.Phase = Phase.PREP_INDEX_CHECKPOINT; + break; + case Phase.PREP_INDEX_CHECKPOINT: + result.Phase = Phase.PREPARE; + break; + case Phase.IN_PROGRESS: + result.Phase = Phase.WAIT_INDEX_CHECKPOINT; + break; + case Phase.WAIT_INDEX_CHECKPOINT: + result.Phase = Phase.WAIT_FLUSH; + break; + default: + result = base.NextState(start, out barrier); + break; + } + + return result; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs new file mode 100644 index 0000000000..f0cfaf7b87 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Diagnostics; + +namespace Tsavorite.core +{ + /// + /// This task contains logic to orchestrate the index and hybrid log checkpoint in parallel + /// + internal sealed class FullCheckpointSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + readonly TsavoriteKV store; + + public FullCheckpointSMTask(TsavoriteKV store) + { + this.store = store; + } + + /// + public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREP_INDEX_CHECKPOINT: + Debug.Assert(store._indexCheckpoint.IsDefault() && + store._hybridLogCheckpoint.IsDefault()); + var fullCheckpointToken = Guid.NewGuid(); + store._indexCheckpointToken = fullCheckpointToken; + store._hybridLogCheckpointToken = fullCheckpointToken; + store.InitializeIndexCheckpoint(store._indexCheckpointToken); + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); + break; + case Phase.WAIT_FLUSH: + store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); + store._indexCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); + break; + case Phase.PERSISTENCE_CALLBACK: + store.WriteIndexMetaInfo(); + store._indexCheckpoint.Reset(); + break; + } + } + + /// + public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs new file mode 100644 index 0000000000..7c772227ff --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// Hybrid log checkpoint state machine. + /// + internal class HybridLogCheckpointSM : VersionChangeSM + { + /// + /// Construct a new HybridLogCheckpointStateMachine with the given tasks. Does not load any tasks by default. + /// + /// upper limit (inclusive) of the version included + /// The tasks to load onto the state machine + public HybridLogCheckpointSM(long targetVersion, params IStateMachineTask[] tasks) + : base(targetVersion, tasks) { } + + /// + public override SystemState NextState(SystemState start, out bool barrier) + { + barrier = false; + var result = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.IN_PROGRESS: + result.Phase = Phase.WAIT_FLUSH; + break; + case Phase.WAIT_FLUSH: + result.Phase = Phase.PERSISTENCE_CALLBACK; + break; + case Phase.PERSISTENCE_CALLBACK: + result.Phase = Phase.REST; + break; + default: + result = base.NextState(start, out barrier); + break; + } + + return result; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs new file mode 100644 index 0000000000..9e807b22d8 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -0,0 +1,104 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + /// + /// This task is the base class for a checkpoint "backend", which decides how a captured version is + /// persisted on disk. + /// + internal abstract class HybridLogCheckpointSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + protected readonly TsavoriteKV store; + long lastVersion; + + public HybridLogCheckpointSMTask(TsavoriteKV store) + { + this.store = store; + } + + /// + public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREPARE: + lastVersion = store.systemState.Version; + if (store._hybridLogCheckpoint.IsDefault()) + { + store._hybridLogCheckpointToken = Guid.NewGuid(); + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); + } + store._hybridLogCheckpoint.info.version = next.Version; + break; + case Phase.IN_PROGRESS: + store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; + store.CheckpointVersionShift(lastVersion, next.Version); + break; + case Phase.WAIT_FLUSH: + store._hybridLogCheckpoint.info.headAddress = store.hlogBase.HeadAddress; + store._hybridLogCheckpoint.info.nextVersion = next.Version; + break; + case Phase.PERSISTENCE_CALLBACK: + CollectMetadata(next, store); + store.WriteHybridLogMetaInfo(); + store.lastVersion = lastVersion; + break; + case Phase.REST: + store._hybridLogCheckpoint.Dispose(); + var nextTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + store.checkpointTcs.SetResult(new LinkedCheckpointInfo { NextTask = nextTcs.Task }); + store.checkpointTcs = nextTcs; + break; + } + } + + protected static void CollectMetadata(SystemState next, TsavoriteKV store) + { + // Collect object log offsets only after flushes + // are completed + var seg = store.hlog.GetSegmentOffsets(); + if (seg != null) + { + store._hybridLogCheckpoint.info.objectLogSegmentOffsets = new long[seg.Length]; + Array.Copy(seg, store._hybridLogCheckpoint.info.objectLogSegmentOffsets, seg.Length); + } + + // Temporarily block new sessions from starting, which may add an entry to the table and resize the + // dictionary. There should be minimal contention here. + lock (store._activeSessions) + { + List toDelete = null; + + // write dormant sessions to checkpoint + foreach (var kvp in store._activeSessions) + { + kvp.Value.session.AtomicSwitch(next.Version - 1); + if (!kvp.Value.isActive) + { + toDelete ??= new(); + toDelete.Add(kvp.Key); + } + } + + // delete any sessions that ended during checkpoint cycle + if (toDelete != null) + { + foreach (var key in toDelete) + _ = store._activeSessions.Remove(key); + } + } + } + + /// + public virtual void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs new file mode 100644 index 0000000000..847d04fb27 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs @@ -0,0 +1,10 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + public interface IStateMachine : IStateMachineTask + { + public SystemState NextState(SystemState currentState, out bool barrier); + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs new file mode 100644 index 0000000000..3e614e40c7 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + public interface IStateMachineTask + { + /// + /// Called before we move to nextState. All participant threads will be in previousState. + /// + /// + /// + public void GlobalBeforeEnteringState(SystemState nextState, StateMachineDriver stateMachineDriver); + + /// + /// Called after we move to nextState. Participant threads may be in previousState or nextState. + /// + /// + /// + public void GlobalAfterEnteringState(SystemState nextState, StateMachineDriver stateMachineDriver); + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs new file mode 100644 index 0000000000..4ba0f6d762 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// This state machine performs an index checkpoint + /// + internal sealed class IndexSnapshotSM : StateMachineBase + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + /// + /// Create a new IndexSnapshotStateMachine + /// + public IndexSnapshotSM(TsavoriteKV store) + : base(-1, new IndexSnapshotSMTask(store)) + { + } + + /// + public override SystemState NextState(SystemState start, out bool barrier) + { + barrier = false; + var result = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.REST: + result.Phase = Phase.PREP_INDEX_CHECKPOINT; + break; + case Phase.PREP_INDEX_CHECKPOINT: + result.Phase = Phase.WAIT_INDEX_ONLY_CHECKPOINT; + break; + case Phase.WAIT_INDEX_ONLY_CHECKPOINT: + result.Phase = Phase.REST; + break; + default: + throw new TsavoriteException("Invalid Enum Argument"); + } + return result; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs new file mode 100644 index 0000000000..d2103cfb6e --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs @@ -0,0 +1,102 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Collections.Generic; +using System.Threading; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + /// + /// This task performs an index checkpoint. + /// + internal sealed class IndexSnapshotSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + readonly TsavoriteKV store; + + public IndexSnapshotSMTask(TsavoriteKV store) + { + this.store = store; + } + + /// + public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREP_INDEX_CHECKPOINT: + if (store._indexCheckpoint.IsDefault()) + { + store._indexCheckpointToken = Guid.NewGuid(); + store.InitializeIndexCheckpoint(store._indexCheckpointToken); + } + + store._indexCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); + store.TakeIndexFuzzyCheckpoint(); + break; + + case Phase.WAIT_INDEX_CHECKPOINT: + case Phase.WAIT_INDEX_ONLY_CHECKPOINT: + break; + + case Phase.REST: + // If the tail address has already been obtained, because another task on the state machine + // has done so earlier (e.g. FullCheckpoint captures log tail at WAIT_FLUSH), don't update + // the tail address. + if (store.ObtainCurrentTailAddress(ref store._indexCheckpoint.info.finalLogicalAddress)) + store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); + if (!store._indexCheckpoint.IsDefault()) + { + store.WriteIndexMetaInfo(); + store._indexCheckpoint.Reset(); + } + + break; + } + } + + /// + public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + } + + /// + public void OnThreadState( + SystemState current, + SystemState prev, + TsavoriteKV store, + TsavoriteKV.TsavoriteExecutionContext ctx, + TSessionFunctionsWrapper sessionFunctions, + List valueTasks, + CancellationToken token = default) + where TSessionFunctionsWrapper : ISessionEpochControl + { + switch (current.Phase) + { + case Phase.PREP_INDEX_CHECKPOINT: + store.GlobalStateMachineStep(current); + break; + case Phase.WAIT_INDEX_CHECKPOINT: + case Phase.WAIT_INDEX_ONLY_CHECKPOINT: + var notify = store.IsIndexFuzzyCheckpointCompleted(); + notify = notify || !store.SameCycle(ctx, current); + + if (valueTasks != null && !notify) + { + var t = store.IsIndexFuzzyCheckpointCompletedAsync(token); + if (!store.SameCycle(ctx, current)) + notify = true; + else + valueTasks.Add(t); + } + + if (!notify) return; + store.GlobalStateMachineStep(current); + break; + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs new file mode 100644 index 0000000000..9b2a03ca0c --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// Abstract base class for IStateMachine that implements that state machine logic + /// with IStateMachineTasks + /// + internal abstract class StateMachineBase : IStateMachine + { + readonly IStateMachineTask[] tasks; + protected long toVersion; + + /// + /// Construct a new SynchronizationStateMachine with the given tasks. The order of tasks given is the + /// order they are executed on each state machine. + /// + /// The ISynchronizationTasks to run on the state machine + protected StateMachineBase(long toVersion = -1, params IStateMachineTask[] tasks) + { + this.toVersion = toVersion; + this.tasks = tasks; + } + + /// + public abstract SystemState NextState(SystemState start, out bool barrier); + + /// + public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + foreach (var task in tasks) + task.GlobalBeforeEnteringState(next, stateMachineDriver); + } + + /// + public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + foreach (var task in tasks) + task.GlobalAfterEnteringState(next, stateMachineDriver); + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs new file mode 100644 index 0000000000..986e1129e4 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -0,0 +1,173 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System.Collections.Generic; +using System.Diagnostics; +using System.Threading; +using System.Threading.Tasks; +using Microsoft.Extensions.Logging; + +namespace Tsavorite.core +{ + public class StateMachineDriver + { + SystemState systemState; + IStateMachine stateMachine; + readonly List waitingList; + SemaphoreSlim stateMachineCompleted; + // All threads have entered the given state + SemaphoreSlim waitForTransitionIn; + // All threads have exited the given state + SemaphoreSlim waitForTransitionOut; + + readonly LightEpoch epoch; + readonly ILogger logger; + + public StateMachineDriver(LightEpoch epoch, ILogger logger = null) + { + this.epoch = epoch; + this.waitingList = []; + this.logger = logger; + } + + internal void AddToWaitingList(SemaphoreSlim waiter) + { + waitingList.Add(waiter); + } + + public bool Register(IStateMachine stateMachine, CancellationToken token = default) + { + if (Interlocked.CompareExchange(ref this.stateMachine, stateMachine, null) != null) + { + return false; + } + stateMachineCompleted = new SemaphoreSlim(0); + _ = Task.Run(async () => await RunStateMachine(token)); + return true; + } + + public async Task RunAsync(IStateMachine stateMachine, CancellationToken token = default) + { + if (Interlocked.CompareExchange(ref this.stateMachine, stateMachine, null) != null) + { + return false; + } + stateMachineCompleted = new SemaphoreSlim(0); + await RunStateMachine(token); + return true; + } + + public async Task CompleteAsync(CancellationToken token = default) + { + var _stateMachineCompleted = stateMachineCompleted; + if (_stateMachineCompleted != null) + { + await _stateMachineCompleted.WaitAsync(token); + } + } + + /// + /// Steps the global state machine. This will change the current global system state and perform some actions + /// as prescribed by the current state machine. This function has no effect if the current state is not + /// the given expected state. + /// + /// expected current global state + void GlobalStateMachineStep(SystemState expectedState) + { + if (!SystemState.Equal(expectedState, systemState)) + return; + + var nextState = stateMachine.NextState(systemState, out var bumpEpoch); + + stateMachine.GlobalBeforeEnteringState(nextState, this); + + // Write new phase + systemState.Word = nextState.Word; + + // Release waiters for new phase + waitForTransitionOut.Release(int.MaxValue); + + // Write new semaphore + waitForTransitionOut = new SemaphoreSlim(0); + waitForTransitionIn = new SemaphoreSlim(0); + + logger?.LogTrace("Moved to {0}, {1}", nextState.Phase, nextState.Version); + + Debug.Assert(!epoch.ThisInstanceProtected()); + try + { + epoch.Resume(); + epoch.BumpCurrentEpoch(() => MakeTransitionWorker(nextState)); + } + finally + { + epoch.Suspend(); + } + waitingList.Add(waitForTransitionIn); + } + + /// + /// Wait for the state machine to change state out of currentState. + /// + /// + /// + public async Task WaitForStateChange(SystemState currentState) + { + var _waitForTransitionOut = waitForTransitionOut; + if (SystemState.Equal(currentState, systemState)) + { + await _waitForTransitionOut.WaitAsync(); + } + } + + /// + /// Wait for all thread participants to complete currentState. + /// + /// + /// + public async Task WaitForCompletion(SystemState currentState) + { + await WaitForStateChange(currentState); + currentState = systemState; + var _waitForTransitionIn = waitForTransitionIn; + if (SystemState.Equal(currentState, systemState)) + { + await _waitForTransitionIn.WaitAsync(); + } + } + + void MakeTransitionWorker(SystemState nextState) + { + stateMachine.GlobalAfterEnteringState(nextState, this); + waitForTransitionIn.Release(int.MaxValue); + } + + async Task ProcessWaitingListAsync(CancellationToken token = default) + { + foreach (var waiter in waitingList) + { + await waiter.WaitAsync(token); + } + waitingList.Clear(); + } + + async Task RunStateMachine(CancellationToken token = default) + { + try + { + while (systemState.Phase != Phase.REST) + { + GlobalStateMachineStep(systemState); + await ProcessWaitingListAsync(token); + } + } + finally + { + var _stateMachineCompleted = stateMachineCompleted; + stateMachineCompleted = null; + _ = Interlocked.Exchange(ref stateMachine, null); + _stateMachineCompleted.Release(int.MaxValue); + } + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs new file mode 100644 index 0000000000..1a7133da1c --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// A VersionChangeStateMachine orchestrates to capture a version, but does not flush to disk. + /// + internal class VersionChangeSM : StateMachineBase + { + /// + /// Construct a new VersionChangeStateMachine with the given tasks. Does not load any tasks by default. + /// + /// upper limit (inclusive) of the version included + /// The tasks to load onto the state machine + protected VersionChangeSM(long targetVersion = -1, params IStateMachineTask[] tasks) : base(targetVersion, tasks) + { + } + + /// + public override SystemState NextState(SystemState start, out bool barrier) + { + barrier = false; + var nextState = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.REST: + nextState.Phase = Phase.PREPARE; + break; + case Phase.PREPARE: + barrier = true; + nextState.Phase = Phase.IN_PROGRESS; + if (toVersion == -1) toVersion = start.Version + 1; + nextState.Version = toVersion; + break; + case Phase.IN_PROGRESS: + nextState.Phase = Phase.REST; + break; + default: + throw new TsavoriteException("Invalid Enum Argument"); + } + return nextState; + } + } +} \ No newline at end of file From 49b9c16f2d9fec1ac273cdcbfa00df2d5e5c9167 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 24 Feb 2025 14:16:26 -0800 Subject: [PATCH 02/49] cleanup --- .../cs/src/core/Index/Checkpointing/FullCheckpointSM.cs | 5 ++--- .../cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs | 5 ++--- .../core/Index/Checkpointing/HybridLogCheckpointSMTask.cs | 4 +++- .../cs/src/core/Index/Checkpointing/IStateMachine.cs | 2 +- .../cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs | 3 +-- .../cs/src/core/Index/Checkpointing/StateMachineBase.cs | 2 +- .../cs/src/core/Index/Checkpointing/StateMachineDriver.cs | 2 +- .../cs/src/core/Index/Checkpointing/VersionChangeSM.cs | 4 +--- 8 files changed, 12 insertions(+), 15 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs index 0a1fef9350..d04cc9dac6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs @@ -24,9 +24,8 @@ public FullCheckpointSM(TsavoriteKV s { } /// - public override SystemState NextState(SystemState start, out bool barrier) + public override SystemState NextState(SystemState start) { - barrier = false; var result = SystemState.Copy(ref start); switch (start.Phase) { @@ -43,7 +42,7 @@ public override SystemState NextState(SystemState start, out bool barrier) result.Phase = Phase.WAIT_FLUSH; break; default: - result = base.NextState(start, out barrier); + result = base.NextState(start); break; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs index 7c772227ff..501f79152b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs @@ -17,9 +17,8 @@ public HybridLogCheckpointSM(long targetVersion, params IStateMachineTask[] task : base(targetVersion, tasks) { } /// - public override SystemState NextState(SystemState start, out bool barrier) + public override SystemState NextState(SystemState start) { - barrier = false; var result = SystemState.Copy(ref start); switch (start.Phase) { @@ -33,7 +32,7 @@ public override SystemState NextState(SystemState start, out bool barrier) result.Phase = Phase.REST; break; default: - result = base.NextState(start, out barrier); + result = base.NextState(start); break; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 9e807b22d8..040175471b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -36,9 +36,11 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); } store._hybridLogCheckpoint.info.version = next.Version; + store._hybridLogCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); + // Capture begin address before checkpoint starts + store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; break; case Phase.IN_PROGRESS: - store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; store.CheckpointVersionShift(lastVersion, next.Version); break; case Phase.WAIT_FLUSH: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs index 847d04fb27..7b238896a7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs @@ -5,6 +5,6 @@ namespace Tsavorite.core { public interface IStateMachine : IStateMachineTask { - public SystemState NextState(SystemState currentState, out bool barrier); + public SystemState NextState(SystemState currentState); } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs index 4ba0f6d762..b239b43f1d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs @@ -19,9 +19,8 @@ public IndexSnapshotSM(TsavoriteKV st } /// - public override SystemState NextState(SystemState start, out bool barrier) + public override SystemState NextState(SystemState start) { - barrier = false; var result = SystemState.Copy(ref start); switch (start.Phase) { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs index 9b2a03ca0c..670dabd9c5 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs @@ -24,7 +24,7 @@ protected StateMachineBase(long toVersion = -1, params IStateMachineTask[] tasks } /// - public abstract SystemState NextState(SystemState start, out bool barrier); + public abstract SystemState NextState(SystemState start); /// public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 986e1129e4..fb39a9bb92 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -77,7 +77,7 @@ void GlobalStateMachineStep(SystemState expectedState) if (!SystemState.Equal(expectedState, systemState)) return; - var nextState = stateMachine.NextState(systemState, out var bumpEpoch); + var nextState = stateMachine.NextState(systemState); stateMachine.GlobalBeforeEnteringState(nextState, this); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs index 1a7133da1c..7197760f5b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs @@ -18,9 +18,8 @@ protected VersionChangeSM(long targetVersion = -1, params IStateMachineTask[] ta } /// - public override SystemState NextState(SystemState start, out bool barrier) + public override SystemState NextState(SystemState start) { - barrier = false; var nextState = SystemState.Copy(ref start); switch (start.Phase) { @@ -28,7 +27,6 @@ public override SystemState NextState(SystemState start, out bool barrier) nextState.Phase = Phase.PREPARE; break; case Phase.PREPARE: - barrier = true; nextState.Phase = Phase.IN_PROGRESS; if (toVersion == -1) toVersion = start.Version + 1; nextState.Version = toVersion; From d111602002acccd0bd1498ddfbd32491559da4f6 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Tue, 25 Feb 2025 10:19:01 -0800 Subject: [PATCH 03/49] updates --- .../IncrementalSnapshotCheckpointSMTask.cs | 69 ++++++++++++++++ .../Checkpointing/SnapshotCheckpointSMTask.cs | 80 +++++++++++++++++++ .../Index/Checkpointing/StateMachineDriver.cs | 9 ++- ...StreamingSnapshotCheckpointStateMachine.cs | 45 +++++++++++ .../StreamingSnapshotCheckpointTask.cs | 59 ++++++++++++++ .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 14 +++- 6 files changed, 270 insertions(+), 6 deletions(-) create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs new file mode 100644 index 0000000000..aeee5d35e1 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs @@ -0,0 +1,69 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// A Incremental Snapshot makes a copy of only changes that have happened since the last full Snapshot. It is + /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place + /// update performance as it does not advance the readonly marker unnecessarily. + /// + internal sealed class IncrementalSnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + public IncrementalSnapshotCheckpointSMTask(TsavoriteKV store) + : base(store) + { + } + + /// + public override void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREPARE: + store._hybridLogCheckpoint = store._lastSnapshotCheckpoint; + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpoint.prevVersion = next.Version; + break; + case Phase.IN_PROGRESS: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; + case Phase.WAIT_FLUSH: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); + + if (store._hybridLogCheckpoint.deltaLog == null) + { + store._hybridLogCheckpoint.deltaFileDevice = store.checkpointManager.GetDeltaLogDevice(store._hybridLogCheckpointToken); + store._hybridLogCheckpoint.deltaFileDevice.Initialize(-1); + store._hybridLogCheckpoint.deltaLog = new DeltaLog(store._hybridLogCheckpoint.deltaFileDevice, store.hlogBase.LogPageSizeBits, -1); + store._hybridLogCheckpoint.deltaLog.InitializeForWrites(store.hlogBase.bufferPool); + } + + // We are writing delta records outside epoch protection, so callee should be able to + // handle corrupted or unexpected concurrent page changes during the flush, e.g., by + // resuming epoch protection if necessary. Correctness is not affected as we will + // only read safe pages during recovery. + store.hlogBase.AsyncFlushDeltaToDevice( + store.hlogBase.FlushedUntilAddress, + store._hybridLogCheckpoint.info.finalLogicalAddress, + store._lastSnapshotCheckpoint.info.finalLogicalAddress, + store._hybridLogCheckpoint.prevVersion, + store._hybridLogCheckpoint.deltaLog, + out store._hybridLogCheckpoint.flushedSemaphore, + store.ThrottleCheckpointFlushDelayMs); + break; + case Phase.PERSISTENCE_CALLBACK: + CollectMetadata(next, store); + store._hybridLogCheckpoint.info.deltaTailAddress = store._hybridLogCheckpoint.deltaLog.TailAddress; + store.WriteHybridLogIncrementalMetaInfo(store._hybridLogCheckpoint.deltaLog); + store._hybridLogCheckpoint.info.deltaTailAddress = store._hybridLogCheckpoint.deltaLog.TailAddress; + store._lastSnapshotCheckpoint = store._hybridLogCheckpoint.Transfer(); + store._hybridLogCheckpoint.Dispose(); + break; + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs new file mode 100644 index 0000000000..190c5cf44c --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// A Snapshot persists a version by making a copy for every entry of that version separate from the log. It is + /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place + /// update performance as it does not advance the readonly marker unnecessarily. + /// + internal sealed class SnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + public SnapshotCheckpointSMTask(TsavoriteKV store) + : base(store) + { + } + + /// + public override void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREPARE: + store._lastSnapshotCheckpoint.Dispose(); + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpoint.info.useSnapshotFile = 1; + break; + case Phase.WAIT_FLUSH: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); + store._hybridLogCheckpoint.info.snapshotFinalLogicalAddress = store._hybridLogCheckpoint.info.finalLogicalAddress; + + store._hybridLogCheckpoint.snapshotFileDevice = + store.checkpointManager.GetSnapshotLogDevice(store._hybridLogCheckpointToken); + store._hybridLogCheckpoint.snapshotFileObjectLogDevice = + store.checkpointManager.GetSnapshotObjectLogDevice(store._hybridLogCheckpointToken); + store._hybridLogCheckpoint.snapshotFileDevice.Initialize(store.hlogBase.GetSegmentSize()); + store._hybridLogCheckpoint.snapshotFileObjectLogDevice.Initialize(-1); + + // If we are using a NullDevice then storage tier is not enabled and FlushedUntilAddress may be ReadOnlyAddress; get all records in memory. + store._hybridLogCheckpoint.info.snapshotStartFlushedLogicalAddress = store.hlogBase.IsNullDevice ? store.hlogBase.HeadAddress : store.hlogBase.FlushedUntilAddress; + + long startPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.snapshotStartFlushedLogicalAddress); + long endPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.finalLogicalAddress); + if (store._hybridLogCheckpoint.info.finalLogicalAddress > + store.hlog.GetStartLogicalAddress(endPage)) + { + endPage++; + } + + // We are writing pages outside epoch protection, so callee should be able to + // handle corrupted or unexpected concurrent page changes during the flush, e.g., by + // resuming epoch protection if necessary. Correctness is not affected as we will + // only read safe pages during recovery. + store.hlogBase.AsyncFlushPagesToDevice( + startPage, + endPage, + store._hybridLogCheckpoint.info.finalLogicalAddress, + store._hybridLogCheckpoint.info.startLogicalAddress, + store._hybridLogCheckpoint.snapshotFileDevice, + store._hybridLogCheckpoint.snapshotFileObjectLogDevice, + out store._hybridLogCheckpoint.flushedSemaphore, + store.ThrottleCheckpointFlushDelayMs); + break; + case Phase.PERSISTENCE_CALLBACK: + // Set actual FlushedUntil to the latest possible data in main log that is on disk + // If we are using a NullDevice then storage tier is not enabled and FlushedUntilAddress may be ReadOnlyAddress; get all records in memory. + store._hybridLogCheckpoint.info.flushedLogicalAddress = store.hlogBase.IsNullDevice ? store.hlogBase.HeadAddress : store.hlogBase.FlushedUntilAddress; + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._lastSnapshotCheckpoint = store._hybridLogCheckpoint.Transfer(); + break; + default: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index fb39a9bb92..b63019dd98 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -23,9 +23,10 @@ public class StateMachineDriver readonly LightEpoch epoch; readonly ILogger logger; - public StateMachineDriver(LightEpoch epoch, ILogger logger = null) + public StateMachineDriver(LightEpoch epoch, SystemState initialState, ILogger logger = null) { this.epoch = epoch; + this.systemState = initialState; this.waitingList = []; this.logger = logger; } @@ -85,7 +86,7 @@ void GlobalStateMachineStep(SystemState expectedState) systemState.Word = nextState.Word; // Release waiters for new phase - waitForTransitionOut.Release(int.MaxValue); + waitForTransitionOut?.Release(int.MaxValue); // Write new semaphore waitForTransitionOut = new SemaphoreSlim(0); @@ -155,11 +156,11 @@ async Task RunStateMachine(CancellationToken token = default) { try { - while (systemState.Phase != Phase.REST) + do { GlobalStateMachineStep(systemState); await ProcessWaitingListAsync(token); - } + } while (systemState.Phase != Phase.REST); } finally { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs new file mode 100644 index 0000000000..9656df1a11 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// StreamingSnapshot checkpoint state machine. + /// + class StreamingSnapshotCheckpointSM : VersionChangeSM + { + /// + /// Construct a new StreamingSnapshotCheckpointStateMachine, drawing boundary at targetVersion. + /// + /// upper limit (inclusive) of the version included + public StreamingSnapshotCheckpointSM(long targetVersion, IStateMachineTask backend) + : base(targetVersion, backend) + { } + + /// + public override SystemState NextState(SystemState start) + { + var result = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.REST: + result.Phase = Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT; + break; + case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: + result.Phase = Phase.PREPARE; + break; + case Phase.IN_PROGRESS: + result.Phase = Phase.WAIT_FLUSH; + break; + case Phase.WAIT_FLUSH: + result.Phase = Phase.REST; + break; + default: + result = base.NextState(start); + break; + } + + return result; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs new file mode 100644 index 0000000000..076beabfc7 --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.Threading; +using System.Threading.Tasks; + +namespace Tsavorite.core +{ + + /// + /// A Streaming Snapshot persists a version by yielding a stream of key-value pairs that correspond to + /// a consistent snapshot of the database, for the old version (v). Unlike Snapshot, StreamingSnapshot + /// is designed to not require tail growth even during the WAIT_FLUSH phase of checkpointing. Further, + /// it does not require a snapshot of the index. Recovery is achieved by replaying the yielded log + /// of key-value pairs and inserting each record into an empty database. + /// + sealed class StreamingSnapshotCheckpointSMTask : HybridLogCheckpointSMTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + readonly long targetVersion; + + public StreamingSnapshotCheckpointSMTask(long targetVersion, TsavoriteKV store) + : base(store) + { + this.targetVersion = targetVersion; + } + + /// + public override void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpointToken = Guid.NewGuid(); + store._hybridLogCheckpoint.info.version = next.Version; + store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; + store._lastSnapshotCheckpoint.Dispose(); + _ = Task.Run(store.StreamingSnapshotScanPhase1); + break; + case Phase.PREPARE: + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; + case Phase.WAIT_FLUSH: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpoint.flushedSemaphore = new SemaphoreSlim(0); + var finalLogicalAddress = store.hlogBase.GetTailAddress(); + Task.Run(() => store.StreamingSnapshotScanPhase2(finalLogicalAddress)); + break; + default: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index e57be1d8bf..8d6d946f61 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -29,6 +29,7 @@ public partial class TsavoriteKV : Ts internal readonly bool UseReadCache; private readonly ReadCopyOptions ReadCopyOptions; internal readonly int sectorSize; + readonly StateMachineDriver stateMachineDriver; /// /// Number of active entries in hash index (does not correspond to total records, due to hash collisions) @@ -160,6 +161,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun LockTable = new OverflowBucketLockTable(this); RevivificationManager = new(this, isFixedLenReviv, kvSettings.RevivificationSettings, logSettings); + stateMachineDriver = new(epoch, SystemState.Make(Phase.REST, 1), kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); systemState = SystemState.Make(Phase.REST, 1); if (kvSettings.TryRecoverLatest) @@ -200,8 +202,12 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT bool result; if (checkpointType == CheckpointType.FoldOver) { - var backend = new FoldOverCheckpointTask(); - result = StartStateMachine(new FullCheckpointStateMachine(backend, targetVersion)); + var backend = new FoldOverSMTask(this); + var sm = new FullCheckpointSM(this, backend, targetVersion); + result = stateMachineDriver.Register(sm); + + //var backend = new FoldOverCheckpointTask(); + //result = StartStateMachine(new FullCheckpointStateMachine(backend, targetVersion)); } else if (checkpointType == CheckpointType.Snapshot) { @@ -448,6 +454,9 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default token.ThrowIfCancellationRequested(); + await stateMachineDriver.CompleteAsync(token); + return; + /* while (true) { var systemState = this.systemState; @@ -485,6 +494,7 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default await task.ConfigureAwait(false); } } + */ } [MethodImpl(MethodImplOptions.AggressiveInlining)] From 9f838f7d15a32ccb45a345194d9ee3b152973bcd Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Tue, 25 Feb 2025 10:19:09 -0800 Subject: [PATCH 04/49] remove dead code --- .../VersionChangeStateMachine.cs | 38 ------------------- 1 file changed, 38 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs index bdf2b2b23b..fa96beabf6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs @@ -79,44 +79,6 @@ public void OnThreadState( } } - /// - /// The FoldOver task simply sets the read only offset to the current end of the log, so a captured version - /// is immutable and will eventually be flushed to disk. - /// - internal sealed class FoldOverTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV store) - { - if (next.Phase == Phase.REST) - // Before leaving the checkpoint, make sure all previous versions are read-only. - store.hlogBase.ShiftReadOnlyToTail(out _, out _); - } - - /// - public void GlobalAfterEnteringState( - SystemState next, - TsavoriteKV store) - { } - - /// - public void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - } - } - /// /// A VersionChangeStateMachine orchestrates to capture a version, but does not flush to disk. /// From c907987c826a7ebaf19e8e6d75c01c54ed525c16 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Tue, 25 Feb 2025 18:30:47 -0800 Subject: [PATCH 05/49] updates --- .../src/core/Allocator/MallocFixedPageSize.cs | 2 + .../core/Index/Checkpointing/Checkpoint.cs | 52 ++++++++++++++ .../Index/Checkpointing/FoldOverSMTask.cs | 13 +++- .../Index/Checkpointing/FullCheckpointSM.cs | 15 ++--- .../Checkpointing/FullCheckpointSMTask.cs | 13 ++-- ...ndexSnapshotSM.cs => IndexCheckpointSM.cs} | 8 +-- ...shotSMTask.cs => IndexCheckpointSMTask.cs} | 54 +++------------ .../StreamingSnapshotCheckpointTask.cs | 6 +- .../core/Index/Recovery/IndexCheckpoint.cs | 9 ++- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 67 ++----------------- 10 files changed, 106 insertions(+), 133 deletions(-) create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs rename libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/{IndexSnapshotSM.cs => IndexCheckpointSM.cs} (71%) rename libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/{IndexSnapshotSMTask.cs => IndexCheckpointSMTask.cs} (50%) diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs index b2d87eb6f5..4c3e8cca53 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/MallocFixedPageSize.cs @@ -312,6 +312,8 @@ public async ValueTask IsCheckpointCompletedAsync(CancellationToken token = defa s.Release(); } + public SemaphoreSlim GetCheckpointSemaphore() => checkpointSemaphore; + /// /// Public facing persistence API /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs new file mode 100644 index 0000000000..69757d8eca --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; + +namespace Tsavorite.core +{ + static class Checkpoint + { + public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); + var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); + var fullCheckpointTask = new FullCheckpointSMTask(store, guid); + + if (checkpointType == CheckpointType.FoldOver) + { + var backend = new FoldOverSMTask(store); + return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); + } + else if (checkpointType == CheckpointType.Snapshot) + { + var backend = new SnapshotCheckpointSMTask(store); + return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); + } + else + { + throw new TsavoriteException("Invalid checkpoint type"); + } + } + + public static IStateMachine Streaming(TsavoriteKV store, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); + var backend = new StreamingSnapshotCheckpointSMTask(targetVersion, store, guid); + return new StreamingSnapshotCheckpointSM(targetVersion, backend); + } + + public static IStateMachine IndexOnly(TsavoriteKV store, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); + var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); + return new IndexCheckpointSM(targetVersion, indexCheckpointTask); + } + } +} diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index 681a7c490c..e43f947f86 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -32,8 +32,17 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri if (next.Phase != Phase.WAIT_FLUSH) return; - _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); - store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; + try + { + store.epoch.Resume(); + _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); + stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); + store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; + } + finally + { + store.epoch.Suspend(); + } } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs index d04cc9dac6..4fe1608fef 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs @@ -6,21 +6,16 @@ namespace Tsavorite.core /// /// The state machine orchestrates a full checkpoint /// - internal sealed class FullCheckpointSM : HybridLogCheckpointSM - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class FullCheckpointSM : HybridLogCheckpointSM { /// - /// Construct a new FullCheckpointStateMachine to use the given checkpoint backend (either fold-over or snapshot), + /// Construct a new FullCheckpointStateMachine to use the given set of checkpoint tasks, /// drawing boundary at targetVersion. /// - /// A task that encapsulates the logic to persist the checkpoint /// upper limit (inclusive) of the version included - public FullCheckpointSM(TsavoriteKV store, IStateMachineTask checkpointBackend, long targetVersion = -1) : base( - targetVersion, - new FullCheckpointSMTask(store), - new IndexSnapshotSMTask(store), - checkpointBackend) + /// Tasks + public FullCheckpointSM(long targetVersion = -1, params IStateMachineTask[] tasks) + : base(targetVersion, tasks) { } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs index f0cfaf7b87..29d9c1e3d7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs @@ -14,10 +14,11 @@ internal sealed class FullCheckpointSMTask { readonly TsavoriteKV store; - - public FullCheckpointSMTask(TsavoriteKV store) + readonly Guid guid; + public FullCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; + this.guid = guid; } /// @@ -26,12 +27,8 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state switch (next.Phase) { case Phase.PREP_INDEX_CHECKPOINT: - Debug.Assert(store._indexCheckpoint.IsDefault() && - store._hybridLogCheckpoint.IsDefault()); - var fullCheckpointToken = Guid.NewGuid(); - store._indexCheckpointToken = fullCheckpointToken; - store._hybridLogCheckpointToken = fullCheckpointToken; - store.InitializeIndexCheckpoint(store._indexCheckpointToken); + Debug.Assert(store._hybridLogCheckpoint.IsDefault()); + store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); break; case Phase.WAIT_FLUSH: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs similarity index 71% rename from libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs index b239b43f1d..b1223afd97 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs @@ -6,15 +6,13 @@ namespace Tsavorite.core /// /// This state machine performs an index checkpoint /// - internal sealed class IndexSnapshotSM : StateMachineBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + internal sealed class IndexCheckpointSM : StateMachineBase { /// /// Create a new IndexSnapshotStateMachine /// - public IndexSnapshotSM(TsavoriteKV store) - : base(-1, new IndexSnapshotSMTask(store)) + public IndexCheckpointSM(long targetVersion = -1, params IStateMachineTask[] tasks) + : base(targetVersion, tasks) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs similarity index 50% rename from libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs index d2103cfb6e..b1135253d6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexSnapshotSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs @@ -3,6 +3,7 @@ using System; using System.Collections.Generic; +using System.Diagnostics; using System.Threading; using System.Threading.Tasks; @@ -11,15 +12,17 @@ namespace Tsavorite.core /// /// This task performs an index checkpoint. /// - internal sealed class IndexSnapshotSMTask : IStateMachineTask + internal sealed class IndexCheckpointSMTask : IStateMachineTask where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { readonly TsavoriteKV store; + readonly Guid guid; - public IndexSnapshotSMTask(TsavoriteKV store) + public IndexCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; + this.guid = guid; } /// @@ -28,18 +31,16 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state switch (next.Phase) { case Phase.PREP_INDEX_CHECKPOINT: - if (store._indexCheckpoint.IsDefault()) - { - store._indexCheckpointToken = Guid.NewGuid(); - store.InitializeIndexCheckpoint(store._indexCheckpointToken); - } - + Debug.Assert(store._indexCheckpoint.IsDefault()); + store._indexCheckpointToken = guid; + store.InitializeIndexCheckpoint(store._indexCheckpointToken); store._indexCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); store.TakeIndexFuzzyCheckpoint(); break; case Phase.WAIT_INDEX_CHECKPOINT: case Phase.WAIT_INDEX_ONLY_CHECKPOINT: + store.AddIndexCheckpointWaitingList(stateMachineDriver); break; case Phase.REST: @@ -53,7 +54,6 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state store.WriteIndexMetaInfo(); store._indexCheckpoint.Reset(); } - break; } } @@ -62,41 +62,5 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) { } - - /// - public void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - switch (current.Phase) - { - case Phase.PREP_INDEX_CHECKPOINT: - store.GlobalStateMachineStep(current); - break; - case Phase.WAIT_INDEX_CHECKPOINT: - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: - var notify = store.IsIndexFuzzyCheckpointCompleted(); - notify = notify || !store.SameCycle(ctx, current); - - if (valueTasks != null && !notify) - { - var t = store.IsIndexFuzzyCheckpointCompletedAsync(token); - if (!store.SameCycle(ctx, current)) - notify = true; - else - valueTasks.Add(t); - } - - if (!notify) return; - store.GlobalStateMachineStep(current); - break; - } - } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs index 076beabfc7..807954ca03 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs @@ -20,11 +20,13 @@ sealed class StreamingSnapshotCheckpointSMTask { readonly long targetVersion; + readonly Guid guid; - public StreamingSnapshotCheckpointSMTask(long targetVersion, TsavoriteKV store) + public StreamingSnapshotCheckpointSMTask(long targetVersion, TsavoriteKV store, Guid guid) : base(store) { this.targetVersion = targetVersion; + this.guid = guid; } /// @@ -34,7 +36,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri { case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: base.GlobalBeforeEnteringState(next, stateMachineDriver); - store._hybridLogCheckpointToken = Guid.NewGuid(); + store._hybridLogCheckpointToken = guid; store._hybridLogCheckpoint.info.version = next.Version; store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; store._lastSnapshotCheckpoint.Dispose(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs index a528657c20..bd1b3cdebc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs @@ -52,6 +52,12 @@ internal bool IsIndexFuzzyCheckpointCompleted() return completed1 && completed2; } + internal void AddIndexCheckpointWaitingList(StateMachineDriver stateMachineDriver) + { + stateMachineDriver.AddToWaitingList(mainIndexCheckpointSemaphore); + stateMachineDriver.AddToWaitingList(overflowBucketsAllocator.GetCheckpointSemaphore()); + } + internal async ValueTask IsIndexFuzzyCheckpointCompletedAsync(CancellationToken token = default) { // Get tasks first to ensure we have captured the semaphore instances synchronously @@ -72,6 +78,7 @@ internal unsafe void BeginMainIndexCheckpoint(int version, IDevice device, out u { long totalSize = state[version].size * sizeof(HashBucket); numBytesWritten = (ulong)totalSize; + mainIndexCheckpointSemaphore = new SemaphoreSlim(0); if (throttleCheckpointFlushDelayMs >= 0) Task.Run(FlushRunner); @@ -94,7 +101,7 @@ void FlushRunner() uint chunkSize = (uint)(totalSize / numChunks); mainIndexCheckpointCallbackCount = numChunks; - mainIndexCheckpointSemaphore = new SemaphoreSlim(0); + if (throttleCheckpointFlushDelayMs >= 0) throttleIndexCheckpointFlushSemaphore = new SemaphoreSlim(0); HashBucket* start = state[version].tableAligned; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 8d6d946f61..b9e5686f72 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -198,35 +198,21 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun /// public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { + IStateMachine stateMachine; token = default; - bool result; - if (checkpointType == CheckpointType.FoldOver) - { - var backend = new FoldOverSMTask(this); - var sm = new FullCheckpointSM(this, backend, targetVersion); - result = stateMachineDriver.Register(sm); - //var backend = new FoldOverCheckpointTask(); - //result = StartStateMachine(new FullCheckpointStateMachine(backend, targetVersion)); - } - else if (checkpointType == CheckpointType.Snapshot) - { - var backend = new SnapshotCheckpointTask(); - result = StartStateMachine(new FullCheckpointStateMachine(backend, targetVersion)); - } - else if (checkpointType == CheckpointType.StreamingSnapshot) + if (checkpointType == CheckpointType.StreamingSnapshot) { if (streamingSnapshotIteratorFunctions is null) throw new TsavoriteException("StreamingSnapshot checkpoint requires a streaming snapshot iterator"); this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; - result = StartStateMachine(new StreamingSnapshotCheckpointStateMachine(targetVersion)); + stateMachine = Checkpoint.Streaming(this, targetVersion, out token); } else - throw new TsavoriteException("Unsupported full checkpoint type"); - - if (result) - token = _hybridLogCheckpointToken; - return result; + { + stateMachine = Checkpoint.Full(this, checkpointType, targetVersion, out token); + } + return stateMachineDriver.Register(stateMachine); } /// @@ -456,45 +442,6 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default await stateMachineDriver.CompleteAsync(token); return; - /* - while (true) - { - var systemState = this.systemState; - if (systemState.Phase == Phase.REST || systemState.Phase == Phase.PREPARE_GROW || - systemState.Phase == Phase.IN_PROGRESS_GROW) - return; - - List valueTasks = new(); - - try - { - epoch.Resume(); - ThreadStateMachineStep(null, NullSession.Instance, valueTasks, token); - } - catch (Exception) - { - _indexCheckpoint.Reset(); - _hybridLogCheckpoint.Dispose(); - throw; - } - finally - { - epoch.Suspend(); - } - - if (valueTasks.Count == 0) - { - // Note: The state machine will not advance as long as there are active locking sessions. - continue; // we need to re-check loop, so we return only when we are at REST - } - - foreach (var task in valueTasks) - { - if (!task.IsCompleted) - await task.ConfigureAwait(false); - } - } - */ } [MethodImpl(MethodImplOptions.AggressiveInlining)] From e715503a0c1ed018c116d25110041f84d854b20a Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 13:48:36 -0800 Subject: [PATCH 06/49] updates --- .../Index/Checkpointing/FoldOverSMTask.cs | 3 +- .../IStateMachineCallback.cs | 6 +- .../core/Index/Checkpointing/IndexResizeSM.cs | 40 ++++++++++ .../Index/Checkpointing/IndexResizeSMTask.cs | 63 +++++++++++++++ .../Index/Checkpointing/StateMachineDriver.cs | 27 ++++++- .../StateTransitions.cs | 0 ...ne.cs => StreamingSnapshotCheckpointSM.cs} | 0 ...s => StreamingSnapshotCheckpointSMTask.cs} | 5 +- .../StreamingSnapshotTsavoriteKV.cs | 19 ----- .../TsavoriteStateMachine.cs | 15 +--- .../Tsavorite/Implementation/SplitIndex.cs | 76 ++++++++++++------- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 34 ++------- .../Tsavorite/cs/test/RecoveryChecks.cs | 3 +- .../Tsavorite/cs/test/StateMachineTests.cs | 6 +- 14 files changed, 196 insertions(+), 101 deletions(-) rename libs/storage/Tsavorite/cs/src/core/Index/{Synchronization => Checkpointing}/IStateMachineCallback.cs (51%) create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs create mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs rename libs/storage/Tsavorite/cs/src/core/Index/{Synchronization => Checkpointing}/StateTransitions.cs (100%) rename libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/{StreamingSnapshotCheckpointStateMachine.cs => StreamingSnapshotCheckpointSM.cs} (100%) rename libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/{StreamingSnapshotCheckpointTask.cs => StreamingSnapshotCheckpointSMTask.cs} (91%) rename libs/storage/Tsavorite/cs/src/core/Index/{Synchronization => Checkpointing}/StreamingSnapshotTsavoriteKV.cs (87%) rename libs/storage/Tsavorite/cs/src/core/Index/{Synchronization => Checkpointing}/TsavoriteStateMachine.cs (93%) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index e43f947f86..22935bd293 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -36,7 +36,8 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri { store.epoch.Resume(); _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); - stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); + if (store._hybridLogCheckpoint.flushedSemaphore != null) + stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; } finally diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IStateMachineCallback.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineCallback.cs similarity index 51% rename from libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IStateMachineCallback.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineCallback.cs index 055a33bb81..5eb3237e26 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IStateMachineCallback.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineCallback.cs @@ -6,13 +6,11 @@ namespace Tsavorite.core /// /// Encapsulates custom logic to be executed as part of Tsavorite's state machine logic /// - public interface IStateMachineCallback - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public interface IStateMachineCallback { /// /// Invoked immediately before every state transition. /// - void BeforeEnteringState(SystemState next, TsavoriteKV tsavorite); + void BeforeEnteringState(SystemState next); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs new file mode 100644 index 0000000000..0bff11bdcd --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// Resizes the index + /// + internal sealed class IndexResizeSM : StateMachineBase + { + /// + /// Constructs a new IndexResizeStateMachine + /// + public IndexResizeSM(int targetVersion = -1, params IStateMachineTask[] tasks) : base(targetVersion, tasks) + { + } + + /// + public override SystemState NextState(SystemState start) + { + var nextState = SystemState.Copy(ref start); + switch (start.Phase) + { + case Phase.REST: + nextState.Phase = Phase.PREPARE_GROW; + break; + case Phase.PREPARE_GROW: + nextState.Phase = Phase.IN_PROGRESS_GROW; + break; + case Phase.IN_PROGRESS_GROW: + nextState.Phase = Phase.REST; + break; + default: + throw new TsavoriteException("Invalid Enum Argument"); + } + + return nextState; + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs new file mode 100644 index 0000000000..4cccbd893e --- /dev/null +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +namespace Tsavorite.core +{ + /// + /// Resizes an index + /// + internal sealed class IndexResizeSMTask : IStateMachineTask + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + readonly TsavoriteKV store; + + public IndexResizeSMTask(TsavoriteKV store) + { + this.store = store; + } + + /// + public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.PREPARE_GROW: + break; + case Phase.IN_PROGRESS_GROW: + // Set up the transition to new version of HT + var numChunks = (int)(store.state[store.resizeInfo.version].size / Constants.kSizeofChunk); + if (numChunks == 0) numChunks = 1; // at least one chunk + + store.numPendingChunksToBeSplit = numChunks; + store.splitStatus = new long[numChunks]; + store.overflowBucketsAllocatorResize = store.overflowBucketsAllocator; + store.overflowBucketsAllocator = new MallocFixedPageSize(); + + // Because version is 0 or 1, indexing by [1 - resizeInfo.version] references to the "new version". + // Once growth initialization is complete, the state versions are swapped by setting resizeInfo.version = 1 - resizeInfo.version. + // Initialize the new version to twice the size of the old version. + store.Initialize(1 - store.resizeInfo.version, store.state[store.resizeInfo.version].size * 2, store.sectorSize); + + store.resizeInfo.version = 1 - store.resizeInfo.version; + break; + case Phase.REST: + // nothing to do + break; + default: + throw new TsavoriteException("Invalid Enum Argument"); + } + } + + /// + public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) + { + switch (next.Phase) + { + case Phase.IN_PROGRESS_GROW: + store.SplitAllBuckets(); + break; + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index b63019dd98..cb8bb2f4eb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; @@ -19,7 +20,7 @@ public class StateMachineDriver SemaphoreSlim waitForTransitionIn; // All threads have exited the given state SemaphoreSlim waitForTransitionOut; - + List callbacks; readonly LightEpoch epoch; readonly ILogger logger; @@ -47,6 +48,18 @@ public bool Register(IStateMachine stateMachine, CancellationToken token = defau return true; } + /// + /// Registers the given callback to be invoked for every state machine transition. Not safe to call with + /// concurrent Tsavorite operations. Excessive synchronization or expensive computation in the callback + /// may slow or halt state machine execution. For advanced users only. + /// + /// callback to register + public void UnsafeRegisterCallback(IStateMachineCallback callback) + { + callbacks ??= new(); + callbacks.Add(callback); + } + public async Task RunAsync(IStateMachine stateMachine, CancellationToken token = default) { if (Interlocked.CompareExchange(ref this.stateMachine, stateMachine, null) != null) @@ -82,6 +95,13 @@ void GlobalStateMachineStep(SystemState expectedState) stateMachine.GlobalBeforeEnteringState(nextState, this); + // Execute any additional registered callbacks + if (callbacks != null) + { + foreach (var callback in callbacks) + callback.BeforeEnteringState(nextState); + } + // Write new phase systemState.Word = nextState.Word; @@ -162,6 +182,11 @@ async Task RunStateMachine(CancellationToken token = default) await ProcessWaitingListAsync(token); } while (systemState.Phase != Phase.REST); } + catch (Exception e) + { + logger?.LogError(e, "Exception in state machine"); + throw; + } finally { var _stateMachineCompleted = stateMachineCompleted; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StateTransitions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs similarity index 100% rename from libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StateTransitions.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs similarity index 100% rename from libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointStateMachine.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs similarity index 91% rename from libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index 807954ca03..73770d48c6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -40,7 +40,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._hybridLogCheckpoint.info.version = next.Version; store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; store._lastSnapshotCheckpoint.Dispose(); - _ = Task.Run(store.StreamingSnapshotScanPhase1); + store.StreamingSnapshotScanPhase1(); break; case Phase.PREPARE: store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); @@ -48,9 +48,8 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri break; case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); - store._hybridLogCheckpoint.flushedSemaphore = new SemaphoreSlim(0); var finalLogicalAddress = store.hlogBase.GetTailAddress(); - Task.Run(() => store.StreamingSnapshotScanPhase2(finalLogicalAddress)); + store.StreamingSnapshotScanPhase2(finalLogicalAddress); break; default: base.GlobalBeforeEnteringState(next, stateMachineDriver); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotTsavoriteKV.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs similarity index 87% rename from libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotTsavoriteKV.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs index 72783c0b3e..908c1a309c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotTsavoriteKV.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs @@ -79,14 +79,6 @@ internal void StreamingSnapshotScanPhase1() logger?.LogError(e, "Exception in StreamingSnapshotScanPhase1"); throw; } - finally - { - // We started this task before entering PREP_STREAMING_SNAPSHOT_CHECKPOINT, so we - // need to wait until the state machine is in PREP_STREAMING_SNAPSHOT_CHECKPOINT - while (systemState.Phase != Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT) - Thread.Yield(); - GlobalStateMachineStep(systemState); - } } class ScanPhase2Functions : IScanIteratorFunctions @@ -145,17 +137,6 @@ internal void StreamingSnapshotScanPhase2(long untilAddress) logger?.LogError(e, "Exception in StreamingSnapshotScanPhase2"); throw; } - finally - { - // Release the semaphore to allow the checkpoint waiting task to proceed - _hybridLogCheckpoint.flushedSemaphore.Release(); - - // We started this task before entering WAIT_FLUSH, so we - // need to wait until the state machine is in WAIT_FLUSH - while (systemState.Phase != Phase.WAIT_FLUSH) - Thread.Yield(); - GlobalStateMachineStep(systemState); - } } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/TsavoriteStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs similarity index 93% rename from libs/storage/Tsavorite/cs/src/core/Index/Synchronization/TsavoriteStateMachine.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs index 3a130db734..0dbabf7a34 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/TsavoriteStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs @@ -23,7 +23,7 @@ public partial class TsavoriteKV : Ts // The current state machine in the system. The value could be stale and point to the previous state machine // if no state machine is active at this time. private ISynchronizationStateMachine currentSyncStateMachine; - private List> callbacks = new(); + //private List> callbacks = new(); internal long lastVersion; /// @@ -62,15 +62,6 @@ public partial class TsavoriteKV : Ts /// public long CurrentVersion => systemState.Version; - /// - /// Registers the given callback to be invoked for every state machine transition. Not safe to call with - /// concurrent Tsavorite operations. Note that registered callbacks execute as part of the critical - /// section of Tsavorite's state transitions. Excessive synchronization or expensive computation in the callback - /// may slow or halt state machine execution. For advanced users only. - /// - /// callback to register - public void UnsafeRegisterCallback(IStateMachineCallback callback) => callbacks.Add(callback); - /// /// Attempt to start the given state machine in the system if no other state machine is active. /// @@ -128,8 +119,8 @@ void MakeTransitionWorker(SystemState intermediate, SystemState nextState) // Execute custom task logic currentSyncStateMachine.GlobalBeforeEnteringState(nextState, this); // Execute any additional callbacks in critical section - foreach (var callback in callbacks) - callback.BeforeEnteringState(nextState, this); + //foreach (var callback in callbacks) + // callback.BeforeEnteringState(nextState, this); var success = MakeTransition(intermediate, nextState); // Guaranteed to succeed, because other threads will always block while the system is in intermediate. diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs index a789e668b1..8e823f99b4 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/SplitIndex.cs @@ -1,6 +1,7 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System.Diagnostics; using System.Threading; namespace Tsavorite.core @@ -9,6 +10,22 @@ public unsafe partial class TsavoriteKV where TAllocator : IAllocator { + internal void SplitAllBuckets() + { + var numChunks = (int)(state[1 - resizeInfo.version].size / Constants.kSizeofChunk); + if (numChunks == 0) numChunks = 1; // at least one chunk + for (var i = 0; i < numChunks; i++) + { + _ = SplitSingleBucket(i, numChunks); + } + + // Splits done, GC the old version of the hash table + Debug.Assert(numPendingChunksToBeSplit == 0); + state[1 - resizeInfo.version] = default; + overflowBucketsAllocatorResize.Dispose(); + overflowBucketsAllocatorResize = null; + } + private void SplitBuckets(long hash) { long masked_bucket_index = hash & state[1 - resizeInfo.version].size_mask; @@ -30,36 +47,8 @@ private void SplitBuckets(int chunkOffset) // Process each chunk, wrapping around the end of the chunk list. for (int i = chunkOffset; i < chunkOffset + numChunks; i++) { - // Try to gain exclusive access to this chunk's split state (1 means locked, 0 means unlocked, 2 means the chunk was already split); - // another thread could also be running SplitChunks. TODO change these numbers to named consts. - if (0 == Interlocked.CompareExchange(ref splitStatus[i & (numChunks - 1)], 1, 0)) + if (SplitSingleBucket(i, numChunks)) { - // "Chunks" are offsets into one contiguous allocation: tableAligned - long chunkSize = state[1 - resizeInfo.version].size / numChunks; - long ptr = chunkSize * (i & (numChunks - 1)); - - HashBucket* src_start = state[1 - resizeInfo.version].tableAligned + ptr; - - // The start of the destination chunk - HashBucket* dest_start0 = state[resizeInfo.version].tableAligned + ptr; - // The midpoint of the destination chunk (old version size is half the new version size) - HashBucket* dest_start1 = state[resizeInfo.version].tableAligned + state[1 - resizeInfo.version].size + ptr; - - SplitChunk(src_start, dest_start0, dest_start1, chunkSize); - - // The split for this chunk is done (2 means completed; no Interlock is needed here because we have exclusive access). - // splitStatus is re-created for each index size operation, so we never need to reset this to zero. - splitStatus[i & (numChunks - 1)] = 2; - - if (Interlocked.Decrement(ref numPendingChunksToBeSplit) == 0) - { - // There are no more chunks to be split so GC the old version of the hash table - state[1 - resizeInfo.version] = default; - overflowBucketsAllocatorResize.Dispose(); - overflowBucketsAllocatorResize = null; - GlobalStateMachineStep(systemState); - return; - } break; } } @@ -70,6 +59,35 @@ private void SplitBuckets(int chunkOffset) } } + private bool SplitSingleBucket(int i, int numChunks) + { + // Try to gain exclusive access to this chunk's split state (1 means locked, 0 means unlocked, 2 means the chunk was already split); + // another thread could also be running SplitChunks. TODO change these numbers to named consts. + if (0 == Interlocked.CompareExchange(ref splitStatus[i & (numChunks - 1)], 1, 0)) + { + // "Chunks" are offsets into one contiguous allocation: tableAligned + long chunkSize = state[1 - resizeInfo.version].size / numChunks; + long ptr = chunkSize * (i & (numChunks - 1)); + + HashBucket* src_start = state[1 - resizeInfo.version].tableAligned + ptr; + + // The start of the destination chunk + HashBucket* dest_start0 = state[resizeInfo.version].tableAligned + ptr; + // The midpoint of the destination chunk (old version size is half the new version size) + HashBucket* dest_start1 = state[resizeInfo.version].tableAligned + state[1 - resizeInfo.version].size + ptr; + + SplitChunk(src_start, dest_start0, dest_start1, chunkSize); + + // The split for this chunk is done (2 means completed; no Interlock is needed here because we have exclusive access). + // splitStatus is re-created for each index size operation, so we never need to reset this to zero. + splitStatus[i & (numChunks - 1)] = 2; + + _ = Interlocked.Decrement(ref numPendingChunksToBeSplit); + return true; + } + return false; + } + private void SplitChunk( HashBucket* _src_start, HashBucket* _dest_start0, diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index b9e5686f72..a7adfc16f6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -29,7 +29,7 @@ public partial class TsavoriteKV : Ts internal readonly bool UseReadCache; private readonly ReadCopyOptions ReadCopyOptions; internal readonly int sectorSize; - readonly StateMachineDriver stateMachineDriver; + internal readonly StateMachineDriver stateMachineDriver; /// /// Number of active entries in hash index (does not correspond to total records, due to hash collisions) @@ -578,37 +578,15 @@ internal Status ContextDelete /// Grow the hash index by a factor of two. Caller should take a full checkpoint after growth, for persistence. /// - /// Whether the grow completed - public bool GrowIndex() + /// Whether the grow completed successfully + public async Task GrowIndexAsync() { if (epoch.ThisInstanceProtected()) throw new TsavoriteException("Cannot use GrowIndex when using non-async sessions"); - if (!StartStateMachine(new IndexResizeStateMachine())) - return false; - - epoch.Resume(); - - try - { - while (true) - { - var _systemState = SystemState.Copy(ref systemState); - if (_systemState.Phase == Phase.PREPARE_GROW) - ThreadStateMachineStep(null, NullSession.Instance, default); - else if (_systemState.Phase == Phase.IN_PROGRESS_GROW) - SplitBuckets(0); - else if (_systemState.Phase == Phase.REST) - break; - epoch.ProtectAndDrain(); - _ = Thread.Yield(); - } - } - finally - { - epoch.Suspend(); - } - return true; + var indexResizeTask = new IndexResizeSMTask(this); + var indexResizeSM = new IndexResizeSM(-1, indexResizeTask); + return await stateMachineDriver.RunAsync(indexResizeSM); } /// diff --git a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs index 4b7c1cd6c3..abde942ea1 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs @@ -734,7 +734,8 @@ public async ValueTask RecoveryCheck5( _ = bc1.CompletePending(true); } - _ = store1.GrowIndex(); + var result = store1.GrowIndexAsync().GetAwaiter().GetResult(); + ClassicAssert.IsTrue(result); for (long key = 0; key < 1000; key++) { diff --git a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs index 30e1960c80..efad4c8d9f 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs @@ -446,7 +446,7 @@ public void LUCScenario3() public void StateMachineCallbackTest1() { var callback = new TestCallback(); - store.UnsafeRegisterCallback(callback); + store.stateMachineDriver.UnsafeRegisterCallback(callback); Prepare(out _, out var s1, out var uc1, out var s2); // We should be in PREPARE, 1 @@ -618,11 +618,11 @@ public override void ReadCompletionCallback(ref AdId key, ref NumClicks input, r } } - public class TestCallback : IStateMachineCallback + public class TestCallback : IStateMachineCallback { private readonly HashSet invokedStates = []; - public void BeforeEnteringState(SystemState next, TsavoriteKV tsavorite) + public void BeforeEnteringState(SystemState next) { ClassicAssert.IsFalse(invokedStates.Contains(next)); _ = invokedStates.Add(next); From 7529adeba44ca40d4d523fa4bc562be99f18acfd Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 15:55:33 -0800 Subject: [PATCH 07/49] updates --- .../src/core/ClientSession/ClientSession.cs | 3 +- .../core/Index/Checkpointing/Checkpoint.cs | 34 +++++++++++++- .../Index/Checkpointing/FoldOverSMTask.cs | 6 ++- .../HybridLogCheckpointSMTask.cs | 6 ++- .../IncrementalSnapshotCheckpointSMTask.cs | 8 +++- .../Checkpointing/SnapshotCheckpointSMTask.cs | 8 +++- .../Index/Checkpointing/StateMachineDriver.cs | 2 + .../StreamingSnapshotCheckpointSMTask.cs | 2 +- .../Checkpointing/TsavoriteStateMachine.cs | 9 +--- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 45 +++++++++---------- .../core/Index/Tsavorite/TsavoriteThread.cs | 8 +++- 11 files changed, 86 insertions(+), 45 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 486efe368f..75e1803324 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -552,7 +552,8 @@ void IClientSession.AtomicSwitch(long version) /// internal bool IsInPreparePhase() { - return store.SystemState.Phase == Phase.PREPARE || store.SystemState.Phase == Phase.PREPARE_GROW; + var storeState = store.stateMachineDriver.SystemState; + return storeState.Phase == Phase.PREPARE || storeState.Phase == Phase.PREPARE_GROW; } #endregion Other Operations diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 69757d8eca..997c06f6bb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -17,12 +17,12 @@ public static IStateMachine Full(Tsav if (checkpointType == CheckpointType.FoldOver) { - var backend = new FoldOverSMTask(store); + var backend = new FoldOverSMTask(store, guid); return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); } else if (checkpointType == CheckpointType.Snapshot) { - var backend = new SnapshotCheckpointSMTask(store); + var backend = new SnapshotCheckpointSMTask(store, guid); return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); } else @@ -48,5 +48,35 @@ public static IStateMachine IndexOnly var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); return new IndexCheckpointSM(targetVersion, indexCheckpointTask); } + + public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); + + if (checkpointType == CheckpointType.FoldOver) + { + var backend = new FoldOverSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); + } + else if (checkpointType == CheckpointType.Snapshot) + { + var backend = new SnapshotCheckpointSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); + } + else + { + throw new TsavoriteException("Invalid checkpoint type"); + } + } + + public static IStateMachine IncrementalHybridLogOnly(TsavoriteKV store, long targetVersion, Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); + } } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index 22935bd293..e5705cf6c3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// @@ -12,8 +14,8 @@ internal sealed class FoldOverSMTask where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { - public FoldOverSMTask(TsavoriteKV store) - : base(store) + public FoldOverSMTask(TsavoriteKV store, Guid guid) + : base(store, guid) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 040175471b..16890a577b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -17,10 +17,12 @@ internal abstract class HybridLogCheckpointSMTask store; long lastVersion; + readonly Guid guid; - public HybridLogCheckpointSMTask(TsavoriteKV store) + public HybridLogCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; + this.guid = guid; } /// @@ -32,7 +34,7 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv lastVersion = store.systemState.Version; if (store._hybridLogCheckpoint.IsDefault()) { - store._hybridLogCheckpointToken = Guid.NewGuid(); + store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); } store._hybridLogCheckpoint.info.version = next.Version; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs index aeee5d35e1..73ce07bbd6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// @@ -12,8 +14,8 @@ internal sealed class IncrementalSnapshotCheckpointSMTask where TAllocator : IAllocator { - public IncrementalSnapshotCheckpointSMTask(TsavoriteKV store) - : base(store) + public IncrementalSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) + : base(store, guid) { } @@ -54,6 +56,8 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._hybridLogCheckpoint.deltaLog, out store._hybridLogCheckpoint.flushedSemaphore, store.ThrottleCheckpointFlushDelayMs); + if (store._hybridLogCheckpoint.flushedSemaphore != null) + stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); break; case Phase.PERSISTENCE_CALLBACK: CollectMetadata(next, store); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs index 190c5cf44c..3b5c226eff 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System; + namespace Tsavorite.core { /// @@ -12,8 +14,8 @@ internal sealed class SnapshotCheckpointSMTask where TAllocator : IAllocator { - public SnapshotCheckpointSMTask(TsavoriteKV store) - : base(store) + public SnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) + : base(store, guid) { } @@ -63,6 +65,8 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._hybridLogCheckpoint.snapshotFileObjectLogDevice, out store._hybridLogCheckpoint.flushedSemaphore, store.ThrottleCheckpointFlushDelayMs); + if (store._hybridLogCheckpoint.flushedSemaphore != null) + stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); break; case Phase.PERSISTENCE_CALLBACK: // Set actual FlushedUntil to the latest possible data in main log that is on disk diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index cb8bb2f4eb..2b9cb69ec3 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -24,6 +24,8 @@ public class StateMachineDriver readonly LightEpoch epoch; readonly ILogger logger; + public SystemState SystemState => SystemState.Copy(ref systemState); + public StateMachineDriver(LightEpoch epoch, SystemState initialState, ILogger logger = null) { this.epoch = epoch; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index 73770d48c6..c80a881e69 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -23,7 +23,7 @@ sealed class StreamingSnapshotCheckpointSMTask store, Guid guid) - : base(store) + : base(store, guid) { this.targetVersion = targetVersion; this.guid = guid; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs index 0dbabf7a34..478baaf2b6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs @@ -40,7 +40,7 @@ public partial class TsavoriteKV : Ts /// /// Get the current state machine state of the system /// - public SystemState SystemState => systemState; + public SystemState SystemState => stateMachineDriver.SystemState; /// /// Version number of the last checkpointed state @@ -52,15 +52,10 @@ public partial class TsavoriteKV : Ts /// public long IncrementalSnapshotTailAddress => _lastSnapshotCheckpoint.deltaLog?.TailAddress ?? 0; - /// - /// Recovered version number (1 if started from clean slate) - /// - public long RecoveredVersion => systemState.Version; - /// /// Current version number of the store /// - public long CurrentVersion => systemState.Version; + public long CurrentVersion => stateMachineDriver.SystemState.Version; /// /// Attempt to start the given state machine in the system if no other state machine is active. diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index a7adfc16f6..285f050761 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -199,7 +199,6 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { IStateMachine stateMachine; - token = default; if (checkpointType == CheckpointType.StreamingSnapshot) { @@ -294,35 +293,33 @@ public bool TryInitiateIndexCheckpoint(out Guid token) public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkpointType, bool tryIncremental = false, long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { - token = default; - bool result; - if (checkpointType == CheckpointType.FoldOver) - { - var backend = new FoldOverCheckpointTask(); - result = StartStateMachine(new HybridLogCheckpointStateMachine(backend, targetVersion)); - } - else if (checkpointType == CheckpointType.Snapshot) - { - ISynchronizationTask backend; - if (tryIncremental && _lastSnapshotCheckpoint.info.guid != default && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress && !hlog.HasObjectLog) - backend = new IncrementalSnapshotCheckpointTask(); - else - backend = new SnapshotCheckpointTask(); - result = StartStateMachine(new HybridLogCheckpointStateMachine(backend, targetVersion)); - } - else if (checkpointType == CheckpointType.StreamingSnapshot) + IStateMachine stateMachine; + + if (checkpointType == CheckpointType.StreamingSnapshot) { if (streamingSnapshotIteratorFunctions is null) throw new TsavoriteException("StreamingSnapshot checkpoint requires a streaming snapshot iterator"); this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; - result = StartStateMachine(new StreamingSnapshotCheckpointStateMachine(targetVersion)); + stateMachine = Checkpoint.Streaming(this, targetVersion, out token); } else - throw new TsavoriteException("Unsupported hybrid log checkpoint type"); - - if (result) - token = _hybridLogCheckpointToken; - return result; + { + token = _lastSnapshotCheckpoint.info.guid; + var incremental = tryIncremental + && checkpointType == CheckpointType.Snapshot + && token != default + && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress + && !hlog.HasObjectLog; + if (incremental) + { + stateMachine = Checkpoint.IncrementalHybridLogOnly(this, targetVersion, token); + } + else + { + stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, targetVersion, out token); + } + } + return stateMachineDriver.Register(stateMachine); } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 2c1b3e4a73..6c98e9da28 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -19,13 +19,17 @@ internal void InternalRefresh Date: Wed, 26 Feb 2025 16:21:32 -0800 Subject: [PATCH 08/49] update --- .../Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 285f050761..cd2faf9eeb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -251,9 +251,8 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// Whether we could initiate the checkpoint. Use CompleteCheckpointAsync to wait completion. public bool TryInitiateIndexCheckpoint(out Guid token) { - var result = StartStateMachine(new IndexSnapshotStateMachine()); - token = _indexCheckpointToken; - return result; + var stateMachine = Checkpoint.IndexOnly(this, -1, out token); + return stateMachineDriver.Register(stateMachine); } /// From e24720c064a33fc97f352d032b562ca68b31cf7c Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 16:53:22 -0800 Subject: [PATCH 09/49] kill code --- libs/server/Metrics/Info/GarnetInfoMetrics.cs | 2 - .../HybridLogCheckpointSMTask.cs | 2 +- .../Index/Checkpointing/StateMachineDriver.cs | 5 + .../Checkpointing/TsavoriteStateMachine.cs | 217 ------------------ .../src/core/Index/Common/ExecutionContext.cs | 1 - .../Index/Interfaces/ISessionEpochControl.cs | 16 -- .../Interfaces/ISessionFunctionsWrapper.cs | 9 +- .../src/core/Index/Interfaces/NullSession.cs | 18 -- .../cs/src/core/Index/Recovery/Recovery.cs | 9 +- .../Synchronization/FoldOverCheckpointTask.cs | 76 ------ .../FullCheckpointStateMachine.cs | 111 --------- .../HybridLogCheckpointOrchestrationTask.cs | 123 ---------- .../HybridLogCheckpointStateMachine.cs | 53 ----- .../ISynchronizationStateMachine.cs | 166 -------------- .../IncrementalSnapshotCheckpointTask.cs | 106 --------- .../IndexResizeStateMachine.cs | 147 ------------ .../IndexSnapshotStateMachine.cs | 136 ----------- .../Synchronization/SnapshotCheckpointTask.cs | 117 ---------- ...StreamingSnapshotCheckpointStateMachine.cs | 49 ---- .../StreamingSnapshotCheckpointTask.cs | 88 ------- .../VersionChangeStateMachine.cs | 125 ---------- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 1 - .../core/Index/Tsavorite/TsavoriteThread.cs | 3 +- 23 files changed, 17 insertions(+), 1563 deletions(-) delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionEpochControl.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Interfaces/NullSession.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FoldOverCheckpointTask.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointOrchestrationTask.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IncrementalSnapshotCheckpointTask.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/SnapshotCheckpointTask.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointStateMachine.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointTask.cs delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs diff --git a/libs/server/Metrics/Info/GarnetInfoMetrics.cs b/libs/server/Metrics/Info/GarnetInfoMetrics.cs index 196a5ef9e0..d9137b78c5 100644 --- a/libs/server/Metrics/Info/GarnetInfoMetrics.cs +++ b/libs/server/Metrics/Info/GarnetInfoMetrics.cs @@ -207,7 +207,6 @@ private void PopulateStoreStats(StoreWrapper storeWrapper) [ new("CurrentVersion", storeWrapper.store.CurrentVersion.ToString()), new("LastCheckpointedVersion", storeWrapper.store.LastCheckpointedVersion.ToString()), - new("RecoveredVersion", storeWrapper.store.RecoveredVersion.ToString()), new("SystemState", storeWrapper.store.SystemState.ToString()), new("IndexSize", storeWrapper.store.IndexSize.ToString()), new("LogDir", storeWrapper.serverOptions.LogDir), @@ -234,7 +233,6 @@ private void PopulateObjectStoreStats(StoreWrapper storeWrapper) [ new("CurrentVersion", storeWrapper.objectStore.CurrentVersion.ToString()), new("LastCheckpointedVersion", storeWrapper.objectStore.LastCheckpointedVersion.ToString()), - new("RecoveredVersion", storeWrapper.objectStore.RecoveredVersion.ToString()), new("SystemState", storeWrapper.objectStore.SystemState.ToString()), new("IndexSize", storeWrapper.objectStore.IndexSize.ToString()), new("LogDir", storeWrapper.serverOptions.LogDir), diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 16890a577b..8ab60ebeda 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -31,7 +31,7 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv switch (next.Phase) { case Phase.PREPARE: - lastVersion = store.systemState.Version; + lastVersion = next.Version; if (store._hybridLogCheckpoint.IsDefault()) { store._hybridLogCheckpointToken = guid; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 2b9cb69ec3..6dd04aa77e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -34,6 +34,11 @@ public StateMachineDriver(LightEpoch epoch, SystemState initialState, ILogger lo this.logger = logger; } + public void SetSystemState(SystemState state) + { + systemState = SystemState.Copy(ref state); + } + internal void AddToWaitingList(SemaphoreSlim waiter) { waitingList.Add(waiter); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs index 478baaf2b6..660864d7cb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs @@ -14,16 +14,6 @@ public partial class TsavoriteKV : Ts where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { - // The current system state, defined as the combination of a phase and a version number. This value - // is observed by all sessions and a state machine communicates its progress to sessions through - // this value - internal SystemState systemState; - // This flag ensures that only one state machine is active at a given time. - private volatile int stateMachineActive = 0; - // The current state machine in the system. The value could be stale and point to the previous state machine - // if no state machine is active at this time. - private ISynchronizationStateMachine currentSyncStateMachine; - //private List> callbacks = new(); internal long lastVersion; /// @@ -56,212 +46,5 @@ public partial class TsavoriteKV : Ts /// Current version number of the store /// public long CurrentVersion => stateMachineDriver.SystemState.Version; - - /// - /// Attempt to start the given state machine in the system if no other state machine is active. - /// - /// The state machine to start - /// true if the state machine has started, false otherwise - private bool StartStateMachine(ISynchronizationStateMachine stateMachine) - { - // return immediately if there is a state machine under way. - if (Interlocked.CompareExchange(ref stateMachineActive, 1, 0) != 0) return false; - currentSyncStateMachine = stateMachine; - - // No latch required because the above Interlock guards against other tasks starting, and only a new task - // is allowed to change Tsavorite global state from REST - GlobalStateMachineStep(systemState); - return true; - } - - // Atomic transition from expectedState -> nextState - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool MakeTransition(SystemState expectedState, SystemState nextState) - { - if (Interlocked.CompareExchange(ref systemState.Word, nextState.Word, expectedState.Word) != - expectedState.Word) return false; - Debug.WriteLine("Moved to {0}, {1}", nextState.Phase, nextState.Version); - logger?.LogTrace("Moved to {0}, {1}", - (nextState.Phase & Phase.INTERMEDIATE) == 0 ? nextState.Phase : "Intermediate (" + (nextState.Phase & ~Phase.INTERMEDIATE) + ")", - nextState.Version); - return true; - } - - /// - /// Steps the global state machine. This will change the current global system state and perform some actions - /// as prescribed by the current state machine. This function has no effect if the current state is not - /// the given expected state. - /// - /// expected current global state - /// whether we bump the epoch for the final state transition - internal void GlobalStateMachineStep(SystemState expectedState, bool bumpEpoch = false) - { - // Between state transition, temporarily block any concurrent execution thread - // from progressing to prevent perceived inconsistencies - var intermediate = SystemState.MakeIntermediate(expectedState); - if (!MakeTransition(expectedState, intermediate)) return; - - var nextState = currentSyncStateMachine.NextState(expectedState); - - if (bumpEpoch) - epoch.BumpCurrentEpoch(() => MakeTransitionWorker(intermediate, nextState)); - else - MakeTransitionWorker(intermediate, nextState); - } - - void MakeTransitionWorker(SystemState intermediate, SystemState nextState) - { - // Execute custom task logic - currentSyncStateMachine.GlobalBeforeEnteringState(nextState, this); - // Execute any additional callbacks in critical section - //foreach (var callback in callbacks) - // callback.BeforeEnteringState(nextState, this); - - var success = MakeTransition(intermediate, nextState); - // Guaranteed to succeed, because other threads will always block while the system is in intermediate. - Debug.Assert(success); - currentSyncStateMachine.GlobalAfterEnteringState(nextState, this); - - // Mark the state machine done as we exit the state machine. - if (nextState.Phase == Phase.REST) stateMachineActive = 0; - } - - // Given the current global state, return the starting point of the state machine cycle - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static SystemState StartOfCurrentCycle(SystemState currentGlobalState) - { - return currentGlobalState.Phase < Phase.REST - ? SystemState.Make(Phase.REST, currentGlobalState.Version - 1) - : SystemState.Make(Phase.REST, currentGlobalState.Version); - } - - // Given the current thread state and global state, fast forward the thread state to the - // current state machine cycle if needed - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private static SystemState FastForwardToCurrentCycle(SystemState threadState, SystemState targetStartState) - { - if (threadState.Version < targetStartState.Version || - threadState.Version == targetStartState.Version && threadState.Phase < targetStartState.Phase) - { - return targetStartState; - } - - return threadState; - } - - /// - /// Check whether thread is in same cycle compared to current systemState - /// - /// - /// - /// - internal bool SameCycle(TsavoriteExecutionContext ctx, SystemState threadState) - { - if (ctx == null) - { - var _systemState = SystemState.Copy(ref systemState); - SystemState.RemoveIntermediate(ref _systemState); - return StartOfCurrentCycle(threadState).Version == StartOfCurrentCycle(_systemState).Version; - } - return ctx.threadStateMachine == currentSyncStateMachine; - } - - /// - /// Steps the thread's local state machine. Threads catch up to the current global state and performs - /// necessary actions associated with the state as defined by the current state machine - /// - /// null if calling without a context (e.g. waiting on a checkpoint) - /// Tsavorite session. - /// Return list of tasks that caller needs to await, to continue checkpointing - /// Cancellation token - /// - private void ThreadStateMachineStep( - TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - #region Capture current (non-intermediate) system state - var currentTask = currentSyncStateMachine; - var targetState = SystemState.Copy(ref systemState); - SystemState.RemoveIntermediate(ref targetState); - - while (currentSyncStateMachine != currentTask) - { - currentTask = currentSyncStateMachine; - targetState = SystemState.Copy(ref systemState); - SystemState.RemoveIntermediate(ref targetState); - } - #endregion - - var currentState = ctx is null ? targetState : SystemState.Make(ctx.phase, ctx.version); - var targetStartState = StartOfCurrentCycle(targetState); - - // No state machine associated with target, or target is in REST phase: - // we can directly fast forward session to target state - if (currentTask == null || targetState.Phase == Phase.REST) - { - if (ctx is not null) - { - ctx.phase = targetState.Phase; - ctx.version = targetState.Version; - ctx.threadStateMachine = currentTask; - } - return; - } - - #region Jump on and execute current state machine - // We start at either the start point or our previous position in the state machine. - // If we are calling from somewhere other than an execution thread (e.g. waiting on - // a checkpoint to complete on a client app thread), we start at current system state - var threadState = targetState; - - if (ctx is not null) - { - if (ctx.threadStateMachine == currentTask) - { - threadState = currentState; - } - else - { - threadState = targetStartState; - ctx.threadStateMachine = currentTask; - } - } - - var previousState = threadState; - do - { - Debug.Assert( - (threadState.Version < targetState.Version) || - (threadState.Version == targetState.Version && - (threadState.Phase <= targetState.Phase || currentTask is IndexSnapshotStateMachine) - )); - - currentTask.OnThreadEnteringState(threadState, previousState, this, ctx, sessionFunctions, valueTasks, token); - - if (ctx is not null) - { - ctx.phase = threadState.Phase; - ctx.version = threadState.Version; - } - - previousState.Word = threadState.Word; - threadState = currentTask.NextState(threadState); - if (systemState.Word != targetState.Word) - { - var tmp = SystemState.Copy(ref systemState); - if (currentSyncStateMachine == currentTask) - { - targetState = tmp; - SystemState.RemoveIntermediate(ref targetState); - } - } - } while (previousState.Word != targetState.Word); - #endregion - - return; - } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index 10ac1f80a1..60a81452ab 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -29,7 +29,6 @@ internal sealed class TsavoriteExecutionContext public AsyncCountDown pendingReads; public AsyncQueue> readyResponses; public int asyncPendingCount; - public ISynchronizationStateMachine threadStateMachine; internal RevivificationStats RevivificationStats = new(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionEpochControl.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionEpochControl.cs deleted file mode 100644 index 73ec887edf..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionEpochControl.cs +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - /// - /// Provides thread management and callback to checkpoint completion (called state machine). - /// - /// This is broken out into a non-generic base interface to allow the use of - /// in . - internal interface ISessionEpochControl - { - void UnsafeResumeThread(); - void UnsafeSuspendThread(); - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs index d460accd2a..2f7dc84ea0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/ISessionFunctionsWrapper.cs @@ -7,7 +7,7 @@ namespace Tsavorite.core /// Provides thread management and all callbacks. A wrapper for IFunctions and additional methods called by TsavoriteImpl; the wrapped /// IFunctions methods provide additional parameters to support the wrapper functionality, then call through to the user implementations. /// - internal interface ISessionFunctionsWrapper : ISessionEpochControl, IVariableLengthInput + internal interface ISessionFunctionsWrapper : IVariableLengthInput where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { @@ -62,7 +62,12 @@ internal interface ISessionFunctionsWrapper stackCtx); void UnlockTransientExclusive(ref TKey key, ref OperationStackContext stackCtx); void UnlockTransientShared(ref TKey key, ref OperationStackContext stackCtx); - #endregion + #endregion + + #region Epoch control + void UnsafeResumeThread(); + void UnsafeSuspendThread(); + #endregion bool CompletePendingWithOutputs(out CompletedOutputIterator completedOutputs, bool wait = false, bool spinWaitForCommit = false); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/NullSession.cs b/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/NullSession.cs deleted file mode 100644 index 4a923a4ed8..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Interfaces/NullSession.cs +++ /dev/null @@ -1,18 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - struct NullSession : ISessionEpochControl - { - public static readonly NullSession Instance = new(); - - public void UnsafeResumeThread() - { - } - - public void UnsafeSuspendThread() - { - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs index db85d4fa0e..3719e0c8c4 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs @@ -416,7 +416,7 @@ public void Reset() hlogBase.Reset(); // Reset system state - systemState = SystemState.Make(Phase.REST, 1); + stateMachineDriver.SetSystemState(SystemState.Make(Phase.REST, 1)); lastVersion = 0; } @@ -545,7 +545,7 @@ private void DoPostRecovery(IndexCheckpointInfo recoveredICInfo, HybridLogCheckp /// Version to set the store to public void SetVersion(long version) { - systemState = SystemState.Make(Phase.REST, version); + stateMachineDriver.SetSystemState(SystemState.Make(Phase.REST, version)); } /// @@ -557,11 +557,8 @@ public void SetVersion(long version) /// Whether we are recovering to the initial page private bool RecoverToInitialPage(IndexCheckpointInfo recoveredICInfo, HybridLogCheckpointInfo recoveredHLCInfo, out long recoverFromAddress) { - // Ensure active state machine to null - currentSyncStateMachine = null; - // Set new system state after recovery - systemState = SystemState.Make(Phase.REST, recoveredHLCInfo.info.version + 1); + stateMachineDriver.SetSystemState(SystemState.Make(Phase.REST, recoveredHLCInfo.info.version + 1)); if (!recoveredICInfo.IsDefault() && recoveryCountdown != null) { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FoldOverCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FoldOverCheckpointTask.cs deleted file mode 100644 index e8f77ae21e..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FoldOverCheckpointTask.cs +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// A FoldOver checkpoint persists a version by setting the read-only marker past the last entry of that - /// version on the log and waiting until it is flushed to disk. It is simple and fast, but can result - /// in garbage entries on the log, and a slower recovery of performance. - /// - internal sealed class FoldOverCheckpointTask : HybridLogCheckpointOrchestrationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public override void GlobalBeforeEnteringState(SystemState next, - TsavoriteKV store) - { - base.GlobalBeforeEnteringState(next, store); - - if (next.Phase == Phase.PREPARE) - { - store._lastSnapshotCheckpoint.Dispose(); - } - - if (next.Phase == Phase.IN_PROGRESS) - base.GlobalBeforeEnteringState(next, store); - - if (next.Phase != Phase.WAIT_FLUSH) return; - - _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); - store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; - } - - /// - public override void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - { - base.OnThreadState(current, prev, store, ctx, sessionFunctions, valueTasks, token); - - if (current.Phase != Phase.WAIT_FLUSH) return; - - if (ctx is null || !ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush]) - { - var s = store._hybridLogCheckpoint.flushedSemaphore; - - var notify = store.hlogBase.FlushedUntilAddress >= store._hybridLogCheckpoint.info.finalLogicalAddress; - notify = notify || !store.SameCycle(ctx, current) || s == null; - - if (valueTasks != null && !notify) - { - valueTasks.Add(new ValueTask(s.WaitAsync(token).ContinueWith(t => s.Release()))); - } - - if (!notify) return; - - if (ctx is not null) - ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush] = true; - } - - store.epoch.Mark(EpochPhaseIdx.WaitFlush, current.Version); - if (store.epoch.CheckIsComplete(EpochPhaseIdx.WaitFlush, current.Version)) - store.GlobalStateMachineStep(current); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs deleted file mode 100644 index b4333117e7..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs +++ /dev/null @@ -1,111 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// This task contains logic to orchestrate the index and hybrid log checkpoint in parallel - /// - internal sealed class FullCheckpointOrchestrationTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREP_INDEX_CHECKPOINT: - Debug.Assert(store._indexCheckpoint.IsDefault() && - store._hybridLogCheckpoint.IsDefault()); - var fullCheckpointToken = Guid.NewGuid(); - store._indexCheckpointToken = fullCheckpointToken; - store._hybridLogCheckpointToken = fullCheckpointToken; - store.InitializeIndexCheckpoint(store._indexCheckpointToken); - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - break; - case Phase.WAIT_FLUSH: - store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); - store._indexCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); - break; - case Phase.PERSISTENCE_CALLBACK: - store.WriteIndexMetaInfo(); - store._indexCheckpoint.Reset(); - break; - } - } - - /// - public void GlobalAfterEnteringState( - SystemState next, - TsavoriteKV store) - { - } - - /// - public void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - } - } - - /// - /// The state machine orchestrates a full checkpoint - /// - internal sealed class FullCheckpointStateMachine : HybridLogCheckpointStateMachine - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Construct a new FullCheckpointStateMachine to use the given checkpoint backend (either fold-over or snapshot), - /// drawing boundary at targetVersion. - /// - /// A task that encapsulates the logic to persist the checkpoint - /// upper limit (inclusive) of the version included - public FullCheckpointStateMachine(ISynchronizationTask checkpointBackend, long targetVersion = -1) : base( - targetVersion, new VersionChangeTask(), new FullCheckpointOrchestrationTask(), - new IndexSnapshotTask(), checkpointBackend) - { } - - /// - public override SystemState NextState(SystemState start) - { - var result = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.REST: - result.Phase = Phase.PREP_INDEX_CHECKPOINT; - break; - case Phase.PREP_INDEX_CHECKPOINT: - result.Phase = Phase.PREPARE; - break; - case Phase.IN_PROGRESS: - result.Phase = Phase.WAIT_INDEX_CHECKPOINT; - break; - case Phase.WAIT_INDEX_CHECKPOINT: - result.Phase = Phase.WAIT_FLUSH; - break; - default: - result = base.NextState(start); - break; - } - - return result; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointOrchestrationTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointOrchestrationTask.cs deleted file mode 100644 index 762468ad92..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointOrchestrationTask.cs +++ /dev/null @@ -1,123 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// This task is the base class for a checkpoint "backend", which decides how a captured version is - /// persisted on disk. - /// - internal abstract class HybridLogCheckpointOrchestrationTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - private long lastVersion; - /// - public virtual void GlobalBeforeEnteringState(SystemState next, - TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREPARE: - lastVersion = store.systemState.Version; - if (store._hybridLogCheckpoint.IsDefault()) - { - store._hybridLogCheckpointToken = Guid.NewGuid(); - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - } - store._hybridLogCheckpoint.info.version = next.Version; - store._hybridLogCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); - // Capture begin address before checkpoint starts - store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; - break; - case Phase.IN_PROGRESS: - store.CheckpointVersionShift(lastVersion, next.Version); - break; - case Phase.WAIT_FLUSH: - store._hybridLogCheckpoint.info.headAddress = store.hlogBase.HeadAddress; - store._hybridLogCheckpoint.info.nextVersion = next.Version; - break; - case Phase.PERSISTENCE_CALLBACK: - CollectMetadata(next, store); - store.WriteHybridLogMetaInfo(); - store.lastVersion = lastVersion; - break; - case Phase.REST: - store._hybridLogCheckpoint.Dispose(); - var nextTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - store.checkpointTcs.SetResult(new LinkedCheckpointInfo { NextTask = nextTcs.Task }); - store.checkpointTcs = nextTcs; - break; - } - } - - protected static void CollectMetadata(SystemState next, TsavoriteKV store) - { - // Collect object log offsets only after flushes - // are completed - var seg = store.hlog.GetSegmentOffsets(); - if (seg != null) - { - store._hybridLogCheckpoint.info.objectLogSegmentOffsets = new long[seg.Length]; - Array.Copy(seg, store._hybridLogCheckpoint.info.objectLogSegmentOffsets, seg.Length); - } - - // Temporarily block new sessions from starting, which may add an entry to the table and resize the - // dictionary. There should be minimal contention here. - lock (store._activeSessions) - { - List toDelete = null; - - // write dormant sessions to checkpoint - foreach (var kvp in store._activeSessions) - { - kvp.Value.session.AtomicSwitch(next.Version - 1); - if (!kvp.Value.isActive) - { - toDelete ??= new(); - toDelete.Add(kvp.Key); - } - } - - // delete any sessions that ended during checkpoint cycle - if (toDelete != null) - { - foreach (var key in toDelete) - _ = store._activeSessions.Remove(key); - } - } - } - - /// - public virtual void GlobalAfterEnteringState(SystemState next, - TsavoriteKV store) - { - } - - /// - public virtual void OnThreadState( - SystemState current, - SystemState prev, TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - if (current.Phase != Phase.PERSISTENCE_CALLBACK) - return; - - store.epoch.Mark(EpochPhaseIdx.CheckpointCompletionCallback, current.Version); - if (store.epoch.CheckIsComplete(EpochPhaseIdx.CheckpointCompletionCallback, current.Version)) - { - store.storeFunctions.OnCheckpointCompleted(); - store.GlobalStateMachineStep(current); - } - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointStateMachine.cs deleted file mode 100644 index 36cd360610..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/HybridLogCheckpointStateMachine.cs +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - /// - /// Hybrid log checkpoint state machine. - /// - internal class HybridLogCheckpointStateMachine : VersionChangeStateMachine - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Construct a new HybridLogCheckpointStateMachine to use the given checkpoint backend (either fold-over or - /// snapshot), drawing boundary at targetVersion. - /// - /// A task that encapsulates the logic to persist the checkpoint - /// upper limit (inclusive) of the version included - public HybridLogCheckpointStateMachine(ISynchronizationTask checkpointBackend, long targetVersion = -1) - : base(targetVersion, new VersionChangeTask(), checkpointBackend) { } - - /// - /// Construct a new HybridLogCheckpointStateMachine with the given tasks. Does not load any tasks by default. - /// - /// upper limit (inclusive) of the version included - /// The tasks to load onto the state machine - protected HybridLogCheckpointStateMachine(long targetVersion, params ISynchronizationTask[] tasks) - : base(targetVersion, tasks) { } - - /// - public override SystemState NextState(SystemState start) - { - var result = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.IN_PROGRESS: - result.Phase = Phase.WAIT_FLUSH; - break; - case Phase.WAIT_FLUSH: - result.Phase = Phase.PERSISTENCE_CALLBACK; - break; - case Phase.PERSISTENCE_CALLBACK: - result.Phase = Phase.REST; - break; - default: - result = base.NextState(start); - break; - } - - return result; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs deleted file mode 100644 index a3577565ef..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs +++ /dev/null @@ -1,166 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// A state machine defines a serious of actions that changes the system, which requires all sessions to - /// synchronize and agree on certain time points. A full run of the state machine is defined as a cycle - /// starting from REST and ending in REST, and only one state machine can be active at a given time. - /// - internal interface ISynchronizationStateMachine - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Returns the version that we expect this state machine to end up at when back to REST, or -1 if not yet known. - /// - /// The version that we expect this state machine to end up at when back to REST - long ToVersion(); - - /// - /// This function models the transition function of a state machine. - /// - /// The current state of the state machine - /// the next state in this state machine - SystemState NextState(SystemState start); - - /// - /// This function is invoked immediately before the global state machine enters the given state. - /// - void GlobalBeforeEnteringState(SystemState next, - TsavoriteKV tsavorite); - - /// - /// This function is invoked immediately after the global state machine enters the given state. - /// - void GlobalAfterEnteringState(SystemState next, - TsavoriteKV tsavorite); - - /// - /// This function is invoked for every thread when they refresh and observe a given state. - /// - /// Note that the function is not allowed to await when async is set to false. - /// - void OnThreadEnteringState(SystemState current, - SystemState prev, - TsavoriteKV tsavorite, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl; - } - - /// - /// An ISynchronizationTask specifies logic to be run on a state machine, but does not specify a transition - /// function. It is therefore possible to write common logic in an ISynchronizationTask and reuse it across - /// multiple state machines, or to choose the task at runtime and achieve polymorphism in the behavior - /// of a concrete state machine class. - /// - internal interface ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// This function is invoked immediately before the global state machine enters the given state. - /// - void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV tsavorite); - - /// - /// This function is invoked immediately after the global state machine enters the given state. - /// - void GlobalAfterEnteringState( - SystemState next, - TsavoriteKV tsavorite); - - /// - /// This function is invoked for every thread when they refresh and observe a given state. - /// - /// Note that the function is not allowed to await when async is set to false. - /// - void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV tsavorite, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl; - } - - /// - /// Abstract base class for ISynchronizationStateMachine that implements that state machine logic - /// with ISynchronizationTasks - /// - internal abstract class SynchronizationStateMachineBase : ISynchronizationStateMachine - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - private readonly ISynchronizationTask[] tasks; - private long toVersion = -1; - - - /// - /// Construct a new SynchronizationStateMachine with the given tasks. The order of tasks given is the - /// order they are executed on each state machine. - /// - /// The ISynchronizationTasks to run on the state machine - protected SynchronizationStateMachineBase(params ISynchronizationTask[] tasks) - { - this.tasks = tasks; - } - - /// - /// Sets ToVersion for return. Defaults to -1 if not set - /// - /// toVersion - protected void SetToVersion(long v) => toVersion = v; - - /// - public long ToVersion() => toVersion; - - /// - public abstract SystemState NextState(SystemState start); - - /// - public void GlobalBeforeEnteringState(SystemState next, - TsavoriteKV tsavorite) - { - foreach (var task in tasks) - task.GlobalBeforeEnteringState(next, tsavorite); - } - - /// - public void GlobalAfterEnteringState(SystemState next, - TsavoriteKV tsavorite) - { - foreach (var task in tasks) - task.GlobalAfterEnteringState(next, tsavorite); - } - - /// - public void OnThreadEnteringState( - SystemState current, - SystemState prev, - TsavoriteKV tsavorite, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - foreach (var task in tasks) - { - task.OnThreadState(current, prev, tsavorite, ctx, sessionFunctions, valueTasks, token); - } - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IncrementalSnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IncrementalSnapshotCheckpointTask.cs deleted file mode 100644 index 99f09dc2da..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IncrementalSnapshotCheckpointTask.cs +++ /dev/null @@ -1,106 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// A Incremental Snapshot makes a copy of only changes that have happened since the last full Snapshot. It is - /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place - /// update performance as it does not advance the readonly marker unnecessarily. - /// - internal sealed class IncrementalSnapshotCheckpointTask : HybridLogCheckpointOrchestrationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public override void GlobalBeforeEnteringState(SystemState next, TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREPARE: - store._hybridLogCheckpoint = store._lastSnapshotCheckpoint; - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpoint.prevVersion = next.Version; - break; - case Phase.IN_PROGRESS: - base.GlobalBeforeEnteringState(next, store); - break; - case Phase.WAIT_FLUSH: - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); - - if (store._hybridLogCheckpoint.deltaLog == null) - { - store._hybridLogCheckpoint.deltaFileDevice = store.checkpointManager.GetDeltaLogDevice(store._hybridLogCheckpointToken); - store._hybridLogCheckpoint.deltaFileDevice.Initialize(-1); - store._hybridLogCheckpoint.deltaLog = new DeltaLog(store._hybridLogCheckpoint.deltaFileDevice, store.hlogBase.LogPageSizeBits, -1); - store._hybridLogCheckpoint.deltaLog.InitializeForWrites(store.hlogBase.bufferPool); - } - - // We are writing delta records outside epoch protection, so callee should be able to - // handle corrupted or unexpected concurrent page changes during the flush, e.g., by - // resuming epoch protection if necessary. Correctness is not affected as we will - // only read safe pages during recovery. - store.hlogBase.AsyncFlushDeltaToDevice( - store.hlogBase.FlushedUntilAddress, - store._hybridLogCheckpoint.info.finalLogicalAddress, - store._lastSnapshotCheckpoint.info.finalLogicalAddress, - store._hybridLogCheckpoint.prevVersion, - store._hybridLogCheckpoint.deltaLog, - out store._hybridLogCheckpoint.flushedSemaphore, - store.ThrottleCheckpointFlushDelayMs); - break; - case Phase.PERSISTENCE_CALLBACK: - CollectMetadata(next, store); - store._hybridLogCheckpoint.info.deltaTailAddress = store._hybridLogCheckpoint.deltaLog.TailAddress; - store.WriteHybridLogIncrementalMetaInfo(store._hybridLogCheckpoint.deltaLog); - store._hybridLogCheckpoint.info.deltaTailAddress = store._hybridLogCheckpoint.deltaLog.TailAddress; - store._lastSnapshotCheckpoint = store._hybridLogCheckpoint.Transfer(); - store._hybridLogCheckpoint.Dispose(); - break; - } - } - - /// - public override void OnThreadState( - SystemState current, - SystemState prev, TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - { - base.OnThreadState(current, prev, store, ctx, sessionFunctions, valueTasks, token); - - if (current.Phase != Phase.WAIT_FLUSH) return; - - if (ctx is null || !ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush]) - { - var s = store._hybridLogCheckpoint.flushedSemaphore; - - var notify = s != null && s.CurrentCount > 0; - notify = notify || !store.SameCycle(ctx, current) || s == null; - - if (valueTasks != null && !notify) - { - Debug.Assert(s != null); - valueTasks.Add(new ValueTask(s.WaitAsync(token).ContinueWith(t => s.Release()))); - } - - if (!notify) return; - - if (ctx is not null) - ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush] = true; - } - - store.epoch.Mark(EpochPhaseIdx.WaitFlush, current.Version); - if (store.epoch.CheckIsComplete(EpochPhaseIdx.WaitFlush, current.Version)) - store.GlobalStateMachineStep(current); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs deleted file mode 100644 index 503a0e7164..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs +++ /dev/null @@ -1,147 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// Resizes an index - /// - internal sealed class IndexResizeTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - bool allThreadsInPrepareGrow; - - /// - public void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREPARE_GROW: - allThreadsInPrepareGrow = false; - break; - case Phase.IN_PROGRESS_GROW: - // Set up the transition to new version of HT - var numChunks = (int)(store.state[store.resizeInfo.version].size / Constants.kSizeofChunk); - if (numChunks == 0) numChunks = 1; // at least one chunk - - store.numPendingChunksToBeSplit = numChunks; - store.splitStatus = new long[numChunks]; - store.overflowBucketsAllocatorResize = store.overflowBucketsAllocator; - store.overflowBucketsAllocator = new MallocFixedPageSize(); - - // Because version is 0 or 1, indexing by [1 - resizeInfo.version] references to the "new version". - // Once growth initialization is complete, the state versions are swapped by setting resizeInfo.version = 1 - resizeInfo.version. - // Initialize the new version to twice the size of the old version. - store.Initialize(1 - store.resizeInfo.version, store.state[store.resizeInfo.version].size * 2, store.sectorSize); - - store.resizeInfo.version = 1 - store.resizeInfo.version; - break; - case Phase.REST: - // nothing to do - break; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - } - - /// - public void GlobalAfterEnteringState( - SystemState next, - TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREPARE_GROW: - bool isProtected = store.epoch.ThisInstanceProtected(); - if (!isProtected) - store.epoch.Resume(); - try - { - store.epoch.BumpCurrentEpoch(() => allThreadsInPrepareGrow = true); - } - finally - { - if (!isProtected) - store.epoch.Suspend(); - } - break; - case Phase.IN_PROGRESS_GROW: - case Phase.REST: - // nothing to do - break; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - } - - /// - public void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - switch (current.Phase) - { - case Phase.PREPARE_GROW: - // Using bumpEpoch: true allows us to guarantee that when system state proceeds, all threads in prior state - // will see that hlog.NumActiveLockingSessions == 0, ensuring that they can potentially block for the next state. - if (allThreadsInPrepareGrow && store.hlogBase.NumActiveLockingSessions == 0) - store.GlobalStateMachineStep(current, bumpEpoch: true); - break; - - case Phase.IN_PROGRESS_GROW: - case Phase.REST: - return; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - } - } - - /// - /// Resizes the index - /// - internal sealed class IndexResizeStateMachine : SynchronizationStateMachineBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Constructs a new IndexResizeStateMachine - /// - public IndexResizeStateMachine() : base(new IndexResizeTask()) { } - - /// - public override SystemState NextState(SystemState start) - { - var nextState = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.REST: - nextState.Phase = Phase.PREPARE_GROW; - break; - case Phase.PREPARE_GROW: - nextState.Phase = Phase.IN_PROGRESS_GROW; - break; - case Phase.IN_PROGRESS_GROW: - nextState.Phase = Phase.REST; - break; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - - return nextState; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs deleted file mode 100644 index 34e6730bef..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// This task performs an index checkpoint. - /// - internal sealed class IndexSnapshotTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREP_INDEX_CHECKPOINT: - if (store._indexCheckpoint.IsDefault()) - { - store._indexCheckpointToken = Guid.NewGuid(); - store.InitializeIndexCheckpoint(store._indexCheckpointToken); - } - - store._indexCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); - store.TakeIndexFuzzyCheckpoint(); - break; - - case Phase.WAIT_INDEX_CHECKPOINT: - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: - break; - - case Phase.REST: - // If the tail address has already been obtained, because another task on the state machine - // has done so earlier (e.g. FullCheckpoint captures log tail at WAIT_FLUSH), don't update - // the tail address. - if (store.ObtainCurrentTailAddress(ref store._indexCheckpoint.info.finalLogicalAddress)) - store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); - if (!store._indexCheckpoint.IsDefault()) - { - store.WriteIndexMetaInfo(); - store._indexCheckpoint.Reset(); - } - - break; - } - } - - /// - public void GlobalAfterEnteringState( - SystemState next, - TsavoriteKV store) - { - } - - /// - public void OnThreadState( - SystemState current, - SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - switch (current.Phase) - { - case Phase.PREP_INDEX_CHECKPOINT: - store.GlobalStateMachineStep(current); - break; - case Phase.WAIT_INDEX_CHECKPOINT: - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: - var notify = store.IsIndexFuzzyCheckpointCompleted(); - notify = notify || !store.SameCycle(ctx, current); - - if (valueTasks != null && !notify) - { - var t = store.IsIndexFuzzyCheckpointCompletedAsync(token); - if (!store.SameCycle(ctx, current)) - notify = true; - else - valueTasks.Add(t); - } - - if (!notify) return; - store.GlobalStateMachineStep(current); - break; - } - } - } - - /// - /// This state machine performs an index checkpoint - /// - internal sealed class IndexSnapshotStateMachine : SynchronizationStateMachineBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Create a new IndexSnapshotStateMachine - /// - public IndexSnapshotStateMachine() : base(new IndexSnapshotTask()) - { - } - - /// - public override SystemState NextState(SystemState start) - { - var result = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.REST: - result.Phase = Phase.PREP_INDEX_CHECKPOINT; - break; - case Phase.PREP_INDEX_CHECKPOINT: - result.Phase = Phase.WAIT_INDEX_ONLY_CHECKPOINT; - break; - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: - result.Phase = Phase.REST; - break; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - - return result; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/SnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/SnapshotCheckpointTask.cs deleted file mode 100644 index e850e156a1..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/SnapshotCheckpointTask.cs +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// A Snapshot persists a version by making a copy for every entry of that version separate from the log. It is - /// slower and more complex than a foldover, but more space-efficient on the log, and retains in-place - /// update performance as it does not advance the readonly marker unnecessarily. - /// - internal sealed class SnapshotCheckpointTask : HybridLogCheckpointOrchestrationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public override void GlobalBeforeEnteringState(SystemState next, TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREPARE: - store._lastSnapshotCheckpoint.Dispose(); - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpoint.info.useSnapshotFile = 1; - break; - case Phase.WAIT_FLUSH: - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); - store._hybridLogCheckpoint.info.snapshotFinalLogicalAddress = store._hybridLogCheckpoint.info.finalLogicalAddress; - - store._hybridLogCheckpoint.snapshotFileDevice = - store.checkpointManager.GetSnapshotLogDevice(store._hybridLogCheckpointToken); - store._hybridLogCheckpoint.snapshotFileObjectLogDevice = - store.checkpointManager.GetSnapshotObjectLogDevice(store._hybridLogCheckpointToken); - store._hybridLogCheckpoint.snapshotFileDevice.Initialize(store.hlogBase.GetSegmentSize()); - store._hybridLogCheckpoint.snapshotFileObjectLogDevice.Initialize(-1); - - // If we are using a NullDevice then storage tier is not enabled and FlushedUntilAddress may be ReadOnlyAddress; get all records in memory. - store._hybridLogCheckpoint.info.snapshotStartFlushedLogicalAddress = store.hlogBase.IsNullDevice ? store.hlogBase.HeadAddress : store.hlogBase.FlushedUntilAddress; - - long startPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.snapshotStartFlushedLogicalAddress); - long endPage = store.hlogBase.GetPage(store._hybridLogCheckpoint.info.finalLogicalAddress); - if (store._hybridLogCheckpoint.info.finalLogicalAddress > - store.hlog.GetStartLogicalAddress(endPage)) - { - endPage++; - } - - // We are writing pages outside epoch protection, so callee should be able to - // handle corrupted or unexpected concurrent page changes during the flush, e.g., by - // resuming epoch protection if necessary. Correctness is not affected as we will - // only read safe pages during recovery. - store.hlogBase.AsyncFlushPagesToDevice( - startPage, - endPage, - store._hybridLogCheckpoint.info.finalLogicalAddress, - store._hybridLogCheckpoint.info.startLogicalAddress, - store._hybridLogCheckpoint.snapshotFileDevice, - store._hybridLogCheckpoint.snapshotFileObjectLogDevice, - out store._hybridLogCheckpoint.flushedSemaphore, - store.ThrottleCheckpointFlushDelayMs); - break; - case Phase.PERSISTENCE_CALLBACK: - // Set actual FlushedUntil to the latest possible data in main log that is on disk - // If we are using a NullDevice then storage tier is not enabled and FlushedUntilAddress may be ReadOnlyAddress; get all records in memory. - store._hybridLogCheckpoint.info.flushedLogicalAddress = store.hlogBase.IsNullDevice ? store.hlogBase.HeadAddress : store.hlogBase.FlushedUntilAddress; - base.GlobalBeforeEnteringState(next, store); - store._lastSnapshotCheckpoint = store._hybridLogCheckpoint.Transfer(); - break; - default: - base.GlobalBeforeEnteringState(next, store); - break; - } - } - - /// - public override void OnThreadState( - SystemState current, - SystemState prev, TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - { - base.OnThreadState(current, prev, store, ctx, sessionFunctions, valueTasks, token); - - if (current.Phase != Phase.WAIT_FLUSH) return; - - if (ctx is null || !ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush]) - { - var s = store._hybridLogCheckpoint.flushedSemaphore; - - var notify = s != null && s.CurrentCount > 0; - notify = notify || !store.SameCycle(ctx, current) || s == null; - - if (valueTasks != null && !notify) - { - Debug.Assert(s != null); - valueTasks.Add(new ValueTask(s.WaitAsync(token).ContinueWith(t => s.Release()))); - } - - if (!notify) return; - - if (ctx is not null) - ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush] = true; - } - - store.epoch.Mark(EpochPhaseIdx.WaitFlush, current.Version); - if (store.epoch.CheckIsComplete(EpochPhaseIdx.WaitFlush, current.Version)) - store.GlobalStateMachineStep(current); - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointStateMachine.cs deleted file mode 100644 index 29128bc656..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointStateMachine.cs +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -namespace Tsavorite.core -{ - /// - /// StreamingSnapshot checkpoint state machine. - /// - class StreamingSnapshotCheckpointStateMachine : VersionChangeStateMachine - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - /// Construct a new StreamingSnapshotCheckpointStateMachine, drawing boundary at targetVersion. - /// - /// upper limit (inclusive) of the version included - public StreamingSnapshotCheckpointStateMachine(long targetVersion) - : base(targetVersion, - new VersionChangeTask(), - new StreamingSnapshotCheckpointTask(targetVersion)) - { } - - /// - public override SystemState NextState(SystemState start) - { - var result = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.REST: - result.Phase = Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT; - break; - case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: - result.Phase = Phase.PREPARE; - break; - case Phase.IN_PROGRESS: - result.Phase = Phase.WAIT_FLUSH; - break; - case Phase.WAIT_FLUSH: - result.Phase = Phase.REST; - break; - default: - result = base.NextState(start); - break; - } - - return result; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointTask.cs deleted file mode 100644 index 2c932f196b..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/StreamingSnapshotCheckpointTask.cs +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - - /// - /// A Streaming Snapshot persists a version by yielding a stream of key-value pairs that correspond to - /// a consistent snapshot of the database, for the old version (v). Unlike Snapshot, StreamingSnapshot - /// is designed to not require tail growth even during the WAIT_FLUSH phase of checkpointing. Further, - /// it does not require a snapshot of the index. Recovery is achieved by replaying the yielded log - /// of key-value pairs and inserting each record into an empty database. - /// - sealed class StreamingSnapshotCheckpointTask : HybridLogCheckpointOrchestrationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - readonly long targetVersion; - - public StreamingSnapshotCheckpointTask(long targetVersion) - { - this.targetVersion = targetVersion; - } - - /// - public override void GlobalBeforeEnteringState(SystemState next, TsavoriteKV store) - { - switch (next.Phase) - { - case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpointToken = Guid.NewGuid(); - store._hybridLogCheckpoint.info.version = next.Version; - store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; - store._lastSnapshotCheckpoint.Dispose(); - _ = Task.Run(store.StreamingSnapshotScanPhase1); - break; - case Phase.PREPARE: - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - base.GlobalBeforeEnteringState(next, store); - break; - case Phase.WAIT_FLUSH: - base.GlobalBeforeEnteringState(next, store); - store._hybridLogCheckpoint.flushedSemaphore = new SemaphoreSlim(0); - var finalLogicalAddress = store.hlogBase.GetTailAddress(); - Task.Run(() => store.StreamingSnapshotScanPhase2(finalLogicalAddress)); - break; - default: - base.GlobalBeforeEnteringState(next, store); - break; - } - } - - /// - public override void OnThreadState( - SystemState current, - SystemState prev, TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - { - base.OnThreadState(current, prev, store, ctx, sessionFunctions, valueTasks, token); - - if (current.Phase != Phase.WAIT_FLUSH) return; - - if (ctx is null) - { - var s = store._hybridLogCheckpoint.flushedSemaphore; - - var notify = s != null && s.CurrentCount > 0; - notify = notify || !store.SameCycle(ctx, current) || s == null; - - if (valueTasks != null && !notify) - { - Debug.Assert(s != null); - valueTasks.Add(new ValueTask(s.WaitAsync(token).ContinueWith(t => s.Release()))); - } - } - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs deleted file mode 100644 index fa96beabf6..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Synchronization/VersionChangeStateMachine.cs +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.Threading; -using System.Threading.Tasks; - -namespace Tsavorite.core -{ - /// - /// A Version change captures a version on the log by forcing all threads to coordinate a move to the next - /// version. It is used as the basis of many other tasks, which decides what they do with the captured - /// version. - /// - internal sealed class VersionChangeTask : ISynchronizationTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - /// - public void GlobalBeforeEnteringState( - SystemState next, - TsavoriteKV store) - { - } - - /// - public void GlobalAfterEnteringState( - SystemState start, - TsavoriteKV store) - { - } - - /// - public void OnThreadState( - SystemState current, SystemState prev, - TsavoriteKV store, - TsavoriteKV.TsavoriteExecutionContext ctx, - TSessionFunctionsWrapper sessionFunctions, - List valueTasks, - CancellationToken token = default) - where TSessionFunctionsWrapper : ISessionEpochControl - { - switch (current.Phase) - { - case Phase.PREPARE: - if (ctx is not null) - ctx.markers[EpochPhaseIdx.Prepare] = true; - - store.epoch.Mark(EpochPhaseIdx.Prepare, current.Version); - - // Using bumpEpoch: true allows us to guarantee that when system state proceeds, all threads in prior state - // will see that hlog.NumActiveLockingSessions == 0, ensuring that they can potentially block for the next state. - if (store.epoch.CheckIsComplete(EpochPhaseIdx.Prepare, current.Version) && store.hlogBase.NumActiveLockingSessions == 0) - store.GlobalStateMachineStep(current, bumpEpoch: store.CheckpointVersionSwitchBarrier); - break; - case Phase.IN_PROGRESS: - if (ctx != null) - { - // Need to be very careful here as threadCtx is changing - var _ctx = prev.Phase == Phase.IN_PROGRESS ? ctx.prevCtx : ctx; - - if (!_ctx.markers[EpochPhaseIdx.InProgress]) - { - _ = TsavoriteKV.AtomicSwitch(ctx, ctx.prevCtx, _ctx.version); - TsavoriteKV.InitContext(ctx, ctx.prevCtx.sessionID, ctx.prevCtx.sessionName); - - // Has to be prevCtx, not ctx - ctx.prevCtx.markers[EpochPhaseIdx.InProgress] = true; - } - } - - store.epoch.Mark(EpochPhaseIdx.InProgress, current.Version); - if (store.epoch.CheckIsComplete(EpochPhaseIdx.InProgress, current.Version)) - store.GlobalStateMachineStep(current); - break; - case Phase.REST: - break; - } - } - } - - /// - /// A VersionChangeStateMachine orchestrates to capture a version, but does not flush to disk. - /// - internal class VersionChangeStateMachine : SynchronizationStateMachineBase - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - private readonly long targetVersion; - - /// - /// Construct a new VersionChangeStateMachine with the given tasks. Does not load any tasks by default. - /// - /// upper limit (inclusive) of the version included - /// The tasks to load onto the state machine - protected VersionChangeStateMachine(long targetVersion = -1, params ISynchronizationTask[] tasks) : base(tasks) - { - this.targetVersion = targetVersion; - } - - /// - public override SystemState NextState(SystemState start) - { - var nextState = SystemState.Copy(ref start); - switch (start.Phase) - { - case Phase.REST: - nextState.Phase = Phase.PREPARE; - break; - case Phase.PREPARE: - nextState.Phase = Phase.IN_PROGRESS; - SetToVersion(targetVersion == -1 ? start.Version + 1 : targetVersion); - nextState.Version = ToVersion(); - break; - case Phase.IN_PROGRESS: - nextState.Phase = Phase.REST; - break; - default: - throw new TsavoriteException("Invalid Enum Argument"); - } - - return nextState; - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index cd2faf9eeb..c9ebf067e2 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -162,7 +162,6 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun RevivificationManager = new(this, isFixedLenReviv, kvSettings.RevivificationSettings, logSettings); stateMachineDriver = new(epoch, SystemState.Make(Phase.REST, 1), kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); - systemState = SystemState.Make(Phase.REST, 1); if (kvSettings.TryRecoverLatest) { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 6c98e9da28..23b0323891 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -84,7 +84,6 @@ internal static void CopyContext(TsavoriteExecutionCo { dst.phase = src.phase; dst.version = src.version; - dst.threadStateMachine = src.threadStateMachine; dst.markers = src.markers; dst.sessionName = src.sessionName; } @@ -107,7 +106,7 @@ internal bool InternalCompletePending systemState.Phase == Phase.REST; + internal bool InRestPhase() => stateMachineDriver.SystemState.Phase == Phase.REST; #region Complete Pending Requests internal void InternalCompletePendingRequests(TSessionFunctionsWrapper sessionFunctions, From 1a8ba12e3a22bac8da279f0bfd438f0d029d00ab Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 17:31:01 -0800 Subject: [PATCH 10/49] updates --- .../src/core/ClientSession/ClientSession.cs | 12 ++-- .../src/core/ClientSession/IClientSession.cs | 2 - .../ClientSession/ManageClientSessions.cs | 69 +++++++------------ .../HybridLogCheckpointSMTask.cs | 26 ------- .../src/core/Index/Common/ExecutionContext.cs | 25 ++++--- .../cs/src/core/Index/Recovery/Recovery.cs | 13 ---- .../core/Index/Tsavorite/TsavoriteThread.cs | 27 -------- 7 files changed, 45 insertions(+), 129 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 75e1803324..ad1175a454 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -7,6 +7,7 @@ using System.Threading; using System.Threading.Tasks; using Microsoft.Extensions.Logging; +using static System.Collections.Specialized.BitVector32; namespace Tsavorite.core { @@ -133,7 +134,9 @@ public void Dispose() // By the time Dispose is called, we should have no outstanding locks, so can use the BasicContext's sessionFunctions. _ = CompletePending(bContext.sessionFunctions, true); - store.DisposeClientSession(ID, ctx.phase); + + if (store.RevivificationManager.IsEnabled) + MergeRevivificationStatsTo(ref store.RevivificationManager.stats, reset: true); } /// @@ -388,7 +391,7 @@ private async ValueTask WaitForCommitAsync(TSessionFun { token.ThrowIfCancellationRequested(); - if (!ctx.prevCtx.pendingReads.IsEmpty || !ctx.pendingReads.IsEmpty) + if (!ctx.pendingReads.IsEmpty) throw new TsavoriteException("Make sure all async operations issued on this session are awaited and completed first"); // Complete all pending sync operations on session @@ -536,11 +539,6 @@ internal void UnsafeSuspendThread() store.epoch.Suspend(); } - void IClientSession.AtomicSwitch(long version) - { - _ = TsavoriteKV.AtomicSwitch(ctx, ctx.prevCtx, version); - } - /// public void MergeRevivificationStatsTo(ref RevivificationStats to, bool reset) => ctx.MergeRevivificationStatsTo(ref to, reset); diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs index 24c770e3c8..2386b5e116 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs @@ -12,8 +12,6 @@ class SessionInfo internal interface IClientSession { - void AtomicSwitch(long version); - void MergeRevivificationStatsTo(ref RevivificationStats globalStats, bool reset); void ResetRevivificationStats(); diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs index b79c015ef6..4b344be654 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs @@ -30,48 +30,20 @@ public ClientSession(); - InitContext(ctx, sessionID, sessionName); + var ctx = new TsavoriteExecutionContext(sessionID, sessionName); ctx.MergeReadCopyOptions(ReadCopyOptions, readCopyOptions); - var prevCtx = new TsavoriteExecutionContext(); - InitContext(prevCtx, sessionID, sessionName); - prevCtx.version--; - prevCtx.ReadCopyOptions = ctx.ReadCopyOptions; - ctx.prevCtx = prevCtx; - - if (_activeSessions == null) - _ = Interlocked.CompareExchange(ref _activeSessions, new Dictionary(), null); var session = new ClientSession(this, ctx, functions); - lock (_activeSessions) - _activeSessions.Add(sessionID, new SessionInfo { sessionName = sessionName, session = session, isActive = true }); - return session; - } - - /// - /// Dispose session with Tsavorite - /// - /// - /// - /// - internal void DisposeClientSession(int sessionID, Phase sessionPhase) - { - // If a session is disposed during a checkpoint cycle, we mark the session - // as inactive, but wait until the end of checkpoint before disposing it - lock (_activeSessions) + if (RevivificationManager.IsEnabled) { - if (_activeSessions.TryGetValue(sessionID, out SessionInfo sessionInfo)) - { - var session = sessionInfo.session; - if (RevivificationManager.IsEnabled) - session.MergeRevivificationStatsTo(ref RevivificationManager.stats, reset: true); - if (sessionPhase == Phase.REST || sessionPhase == Phase.PREPARE_GROW || sessionPhase == Phase.IN_PROGRESS_GROW) - _ = _activeSessions.Remove(sessionID); - else - sessionInfo.isActive = false; - } + if (_activeSessions == null) + _ = Interlocked.CompareExchange(ref _activeSessions, [], null); + + lock (_activeSessions) + _activeSessions.Add(sessionID, new SessionInfo { sessionName = sessionName, session = session, isActive = true }); } + return session; } /// @@ -79,13 +51,17 @@ internal void DisposeClientSession(int sessionID, Phase sessionPhase) /// public string DumpRevivificationStats() { - lock (_activeSessions) + if (_activeSessions != null) { - // Merge the session-level stats into the global stats, clear the session-level stats, and keep the cumulative stats. - foreach (var sessionInfo in _activeSessions.Values) - sessionInfo.session.MergeRevivificationStatsTo(ref RevivificationManager.stats, reset: true); - return RevivificationManager.stats.Dump(); + lock (_activeSessions) + { + // Merge the session-level stats into the global stats, clear the session-level stats, and keep the cumulative stats. + foreach (var sessionInfo in _activeSessions.Values) + sessionInfo.session.MergeRevivificationStatsTo(ref RevivificationManager.stats, reset: true); + + } } + return RevivificationManager.stats.Dump(); } /// @@ -93,12 +69,15 @@ public string DumpRevivificationStats() /// public void ResetRevivificationStats() { - lock (_activeSessions) + if (_activeSessions != null) { - foreach (var sessionInfo in _activeSessions.Values) - sessionInfo.session.ResetRevivificationStats(); - RevivificationManager.stats.Reset(); + lock (_activeSessions) + { + foreach (var sessionInfo in _activeSessions.Values) + sessionInfo.session.ResetRevivificationStats(); + } } + RevivificationManager.stats.Reset(); } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 8ab60ebeda..a8a55577bc 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -2,7 +2,6 @@ // Licensed under the MIT license. using System; -using System.Collections.Generic; using System.Threading.Tasks; namespace Tsavorite.core @@ -73,31 +72,6 @@ protected static void CollectMetadata(SystemState next, TsavoriteKV toDelete = null; - - // write dormant sessions to checkpoint - foreach (var kvp in store._activeSessions) - { - kvp.Value.session.AtomicSwitch(next.Version - 1); - if (!kvp.Value.isActive) - { - toDelete ??= new(); - toDelete.Add(kvp.Key); - } - } - - // delete any sessions that ended during checkpoint cycle - if (toDelete != null) - { - foreach (var key in toDelete) - _ = store._activeSessions.Remove(key); - } - } } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index 60a81452ab..fdb0c59a1e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -14,24 +14,33 @@ public partial class TsavoriteKV : Ts { internal sealed class TsavoriteExecutionContext { - internal int sessionID; - internal string sessionName; + internal readonly int sessionID; + internal readonly string sessionName; // Control automatic Read copy operations. These flags override flags specified at the TsavoriteKV level, but may be overridden on the individual Read() operations internal ReadCopyOptions ReadCopyOptions; internal long version; public Phase phase; - - public bool[] markers; public long totalPending; - public Dictionary> ioPendingRequests; - public AsyncCountDown pendingReads; - public AsyncQueue> readyResponses; + public readonly Dictionary> ioPendingRequests; + public readonly AsyncCountDown pendingReads; + public readonly AsyncQueue> readyResponses; public int asyncPendingCount; internal RevivificationStats RevivificationStats = new(); + public TsavoriteExecutionContext(int sessionID, string sessionName) + { + phase = Phase.REST; + version = 1; + this.sessionID = sessionID; + this.sessionName = sessionName; + readyResponses = new AsyncQueue>(); + ioPendingRequests = new Dictionary>(); + pendingReads = new AsyncCountDown(); + } + public int SyncIoPendingCount => ioPendingRequests.Count - asyncPendingCount; public bool IsInV1 @@ -84,8 +93,6 @@ public async ValueTask WaitPendingAsync(CancellationToken token = default) } public bool InNewVersion => phase < Phase.REST; - - public TsavoriteExecutionContext prevCtx; } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs index 3719e0c8c4..10798bcf52 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Recovery.cs @@ -1174,19 +1174,6 @@ private void AsyncFlushPageCallbackForRecovery(uint errorCode, uint numBytes, ob result.Free(); } } - - internal static bool AtomicSwitch(TsavoriteExecutionContext fromCtx, TsavoriteExecutionContext toCtx, long version) - { - lock (toCtx) - { - if (toCtx.version < version) - { - CopyContext(fromCtx, toCtx); - return true; - } - } - return false; - } } public abstract partial class AllocatorBase : IDisposable diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 23b0323891..e03970683e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -61,33 +61,6 @@ internal void InternalRefresh(TsavoriteExecutionContext ctx, int sessionID, string sessionName) - { - ctx.phase = Phase.REST; - // The system version starts at 1. Because we do not know what the current state machine state is, - // we need to play it safe and initialize context behind the system state. Otherwise the session may - // never "catch up" with the rest of the system when stepping through the state machine as it is ahead. - ctx.version = 1; - ctx.markers = new bool[8]; - ctx.sessionID = sessionID; - ctx.sessionName = sessionName; - - if (ctx.readyResponses is null) - { - ctx.readyResponses = new AsyncQueue>(); - ctx.ioPendingRequests = new Dictionary>(); - ctx.pendingReads = new AsyncCountDown(); - } - } - - internal static void CopyContext(TsavoriteExecutionContext src, TsavoriteExecutionContext dst) - { - dst.phase = src.phase; - dst.version = src.version; - dst.markers = src.markers; - dst.sessionName = src.sessionName; - } - internal bool InternalCompletePending(TSessionFunctionsWrapper sessionFunctions, bool wait = false, CompletedOutputIterator completedOutputs = null) where TSessionFunctionsWrapper : ISessionFunctionsWrapper From a1b87a929971a20fbb8cc0db7b6c636d7c8236af Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 17:44:18 -0800 Subject: [PATCH 11/49] simplify LightEpoch --- .../cs/src/core/Epochs/LightEpoch.cs | 44 +------------------ 1 file changed, 1 insertion(+), 43 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs index aaff50ab7d..8a5e3022b7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs +++ b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs @@ -274,48 +274,6 @@ public void BumpCurrentEpoch(Action onDrain) ProtectAndDrain(); } - /// - /// Mechanism for threads to mark some activity as completed until - /// some version by this thread - /// - /// ID of activity - /// Version - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void Mark(int markerIdx, long version) - { - Debug.Assert(markerIdx < 6); - (*(tableAligned + Metadata.threadEntryIndex)).markers[markerIdx] = version; - } - - /// - /// Check if all active threads have completed the some - /// activity until given version. - /// - /// ID of activity - /// Version - /// Whether complete - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool CheckIsComplete(int markerIdx, long version) - { - Debug.Assert(markerIdx < 6); - - // check if all threads have reported complete - for (int index = 1; index <= kTableSize; ++index) - { - long entry_epoch = (*(tableAligned + index)).localCurrentEpoch; - long fc_version = (*(tableAligned + index)).markers[markerIdx]; - if (0 != entry_epoch) - { - if ((fc_version != version) && (entry_epoch < long.MaxValue)) - { - return false; - } - } - } - return true; - } - /// /// Looks at all threads and return the latest safe epoch /// @@ -519,7 +477,7 @@ struct Entry public int reentrant; [FieldOffset(16)] - public fixed long markers[6]; + public fixed long placeholder[6]; public override string ToString() => $"lce = {localCurrentEpoch}, tid = {threadId}, re-ent {reentrant}"; } From 69c07e900dc666810e55c7f9008fdd776f4a0ac3 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 17:53:20 -0800 Subject: [PATCH 12/49] move epvs to test --- .../cs/src/core/Epochs/FastThreadLocal.cs | 80 ------------------- .../EpochProtectedVersionScheme.cs | 7 +- 2 files changed, 4 insertions(+), 83 deletions(-) delete mode 100644 libs/storage/Tsavorite/cs/src/core/Epochs/FastThreadLocal.cs rename libs/storage/Tsavorite/cs/{src/core/Epochs => test}/EpochProtectedVersionScheme.cs (98%) diff --git a/libs/storage/Tsavorite/cs/src/core/Epochs/FastThreadLocal.cs b/libs/storage/Tsavorite/cs/src/core/Epochs/FastThreadLocal.cs deleted file mode 100644 index 7a9a9d48e0..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Epochs/FastThreadLocal.cs +++ /dev/null @@ -1,80 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Threading; - -namespace Tsavorite.core -{ - /// - /// Fast implementation of instance-thread-local variables - /// - /// - internal sealed class FastThreadLocal - { - // Max instances supported - private const int kMaxInstances = 128; - - [ThreadStatic] - private static T[] tl_values; - [ThreadStatic] - private static int[] tl_iid; - - private readonly int offset; - private readonly int iid; - - private static readonly int[] instances = new int[kMaxInstances]; - private static int instanceId = 0; - - public FastThreadLocal() - { - iid = Interlocked.Increment(ref instanceId); - - for (int i = 0; i < kMaxInstances; i++) - { - if (0 == Interlocked.CompareExchange(ref instances[i], iid, 0)) - { - offset = i; - return; - } - } - throw new TsavoriteException("Unsupported number of simultaneous instances"); - } - - public void InitializeThread() - { - if (tl_values == null) - { - tl_values = new T[kMaxInstances]; - tl_iid = new int[kMaxInstances]; - } - if (tl_iid[offset] != iid) - { - tl_iid[offset] = iid; - tl_values[offset] = default(T); - } - } - - public void DisposeThread() - { - tl_values[offset] = default(T); - tl_iid[offset] = 0; - } - - /// - /// Dispose instance for all threads - /// - public void Dispose() - { - instances[offset] = 0; - } - - public T Value - { - get => tl_values[offset]; - set => tl_values[offset] = value; - } - - public bool IsInitializedForThread => (tl_values != null) && (iid == tl_iid[offset]); - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Epochs/EpochProtectedVersionScheme.cs b/libs/storage/Tsavorite/cs/test/EpochProtectedVersionScheme.cs similarity index 98% rename from libs/storage/Tsavorite/cs/src/core/Epochs/EpochProtectedVersionScheme.cs rename to libs/storage/Tsavorite/cs/test/EpochProtectedVersionScheme.cs index 79cd391c93..384d6204f7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Epochs/EpochProtectedVersionScheme.cs +++ b/libs/storage/Tsavorite/cs/test/EpochProtectedVersionScheme.cs @@ -6,8 +6,9 @@ using System.Runtime.CompilerServices; using System.Runtime.InteropServices; using System.Threading; +using Tsavorite.core; -namespace Tsavorite.core +namespace Tsavorite.test { /// /// The current state of a state-machine operation such as a checkpoint. @@ -112,7 +113,7 @@ public override string ToString() } /// - /// Compare the current to for equality if obj is also a + /// Compare the current to for equality if obj is also a /// public override bool Equals(object obj) { @@ -126,7 +127,7 @@ public override int GetHashCode() } /// - /// Compare the current to for equality + /// Compare the current to for equality /// private bool Equals(VersionSchemeState other) { From 6ec5233d5a53f3f09b18bc04470184e397834102 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 26 Feb 2025 17:58:16 -0800 Subject: [PATCH 13/49] nits --- .../src/core/Index/Checkpointing/IndexCheckpointSMTask.cs | 3 --- .../src/core/Index/Checkpointing/TsavoriteStateMachine.cs | 7 ------- .../Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs | 1 - 3 files changed, 11 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs index b1135253d6..be80508318 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs @@ -2,10 +2,7 @@ // Licensed under the MIT license. using System; -using System.Collections.Generic; using System.Diagnostics; -using System.Threading; -using System.Threading.Tasks; namespace Tsavorite.core { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs index 660864d7cb..f4f941027a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs @@ -1,13 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System.Collections.Generic; -using System.Diagnostics; -using System.Runtime.CompilerServices; -using System.Threading; -using System.Threading.Tasks; -using Microsoft.Extensions.Logging; - namespace Tsavorite.core { public partial class TsavoriteKV : TsavoriteBase diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index fdb0c59a1e..0e8a8a03e0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -27,7 +27,6 @@ internal sealed class TsavoriteExecutionContext public readonly AsyncCountDown pendingReads; public readonly AsyncQueue> readyResponses; public int asyncPendingCount; - internal RevivificationStats RevivificationStats = new(); public TsavoriteExecutionContext(int sessionID, string sessionName) From 9599202c8f47090fd58fd2cef75f1abb2e588f08 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 14:27:31 -0800 Subject: [PATCH 14/49] updates --- .../Index/Checkpointing/StateMachineDriver.cs | 27 +++++++++++++------ .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 12 +++++++-- .../cs/test/StateMachineBarrierTests.cs | 5 ++++ .../Tsavorite/cs/test/StateMachineTests.cs | 5 ++++ 4 files changed, 39 insertions(+), 10 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 6dd04aa77e..1f16f6acd1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -15,7 +15,7 @@ public class StateMachineDriver SystemState systemState; IStateMachine stateMachine; readonly List waitingList; - SemaphoreSlim stateMachineCompleted; + TaskCompletionSource stateMachineCompleted; // All threads have entered the given state SemaphoreSlim waitForTransitionIn; // All threads have exited the given state @@ -50,8 +50,8 @@ public bool Register(IStateMachine stateMachine, CancellationToken token = defau { return false; } - stateMachineCompleted = new SemaphoreSlim(0); - _ = Task.Run(async () => await RunStateMachine(token)); + stateMachineCompleted = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + _ = Task.Run(async () => await RunStateMachine(token).ConfigureAwait(false)); return true; } @@ -73,18 +73,20 @@ public async Task RunAsync(IStateMachine stateMachine, CancellationToken t { return false; } - stateMachineCompleted = new SemaphoreSlim(0); - await RunStateMachine(token); + stateMachineCompleted = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + await RunStateMachine(token).ConfigureAwait(false); return true; } - public async Task CompleteAsync(CancellationToken token = default) + public async Task CompleteAsync(CancellationToken token = default) { var _stateMachineCompleted = stateMachineCompleted; if (_stateMachineCompleted != null) { - await _stateMachineCompleted.WaitAsync(token); + using var reg = token.Register(() => _stateMachineCompleted.TrySetCanceled()); + return await _stateMachineCompleted.Task.WithCancellationAsync(token).ConfigureAwait(false); } + return false; } /// @@ -181,6 +183,7 @@ async Task ProcessWaitingListAsync(CancellationToken token = default) async Task RunStateMachine(CancellationToken token = default) { + Exception ex = null; try { do @@ -192,6 +195,7 @@ async Task RunStateMachine(CancellationToken token = default) catch (Exception e) { logger?.LogError(e, "Exception in state machine"); + ex = e; throw; } finally @@ -199,7 +203,14 @@ async Task RunStateMachine(CancellationToken token = default) var _stateMachineCompleted = stateMachineCompleted; stateMachineCompleted = null; _ = Interlocked.Exchange(ref stateMachine, null); - _stateMachineCompleted.Release(int.MaxValue); + if (ex != null) + { + _stateMachineCompleted.TrySetException(ex); + } + else + { + _stateMachineCompleted.TrySetResult(true); + } } } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index c9ebf067e2..56bc19bdcf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -434,8 +434,16 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default throw new TsavoriteException("Cannot use CompleteCheckpointAsync when using non-async sessions"); token.ThrowIfCancellationRequested(); - - await stateMachineDriver.CompleteAsync(token); + try + { + await stateMachineDriver.CompleteAsync(token); + } + catch + { + _indexCheckpoint.Reset(); + _hybridLogCheckpoint.Dispose(); + throw; + } return; } diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index 40d194d570..6963ee37b4 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System.IO; +using System.Threading; using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; @@ -143,6 +144,10 @@ void Prepare(out SimpleFunctions f, _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver, targetVersion: toVersion); + // Wait for PREPARE phase + while (!SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)) + Thread.Yield(); + // We should be in PREPARE, 1 ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); } diff --git a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs index efad4c8d9f..2dc0a53064 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs @@ -3,6 +3,7 @@ using System.Collections.Generic; using System.IO; +using System.Threading; using NUnit.Framework; using NUnit.Framework.Legacy; using Tsavorite.core; @@ -571,6 +572,10 @@ void Prepare(out SimpleFunctions f, _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver, targetVersion: toVersion); + // Wait for PREPARE phase + while (!SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)) + Thread.Yield(); + // We should be in PREPARE, 1 ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); } From a24e7e6da200659c565d101fef2915f10bd25438 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 16:09:47 -0800 Subject: [PATCH 15/49] updates --- libs/storage/Tsavorite/cs/test/StateMachineTests.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs index 2dc0a53064..3dc3873a92 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs @@ -15,6 +15,7 @@ namespace Tsavorite.test.statemachine using StructStoreFunctions = StoreFunctions>; [TestFixture] + [Ignore("State machine has changed")] public class StateMachineTests { IDevice log; From 223abf6c7cc1b38886ed96f6c0db192de838f0a1 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 16:46:28 -0800 Subject: [PATCH 16/49] formatting --- .../Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs | 2 +- .../Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs | 2 +- .../cs/src/core/Index/Checkpointing/IStateMachineTask.cs | 2 +- .../cs/src/core/Index/Checkpointing/StateMachineBase.cs | 2 +- .../cs/src/core/Index/Checkpointing/StateMachineDriver.cs | 2 +- .../Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 997c06f6bb..c53b083420 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -79,4 +79,4 @@ public static IStateMachine IncrementalHybridLogOnly public void GlobalAfterEnteringState(SystemState nextState, StateMachineDriver stateMachineDriver); } -} +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs index 670dabd9c5..f1ba3cbe0d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs @@ -40,4 +40,4 @@ public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateM task.GlobalAfterEnteringState(next, stateMachineDriver); } } -} +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 1f16f6acd1..62aa1d9fd1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -214,4 +214,4 @@ async Task RunStateMachine(CancellationToken token = default) } } } -} +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs index bd1b3cdebc..ea0400d606 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/IndexCheckpoint.cs @@ -101,7 +101,7 @@ void FlushRunner() uint chunkSize = (uint)(totalSize / numChunks); mainIndexCheckpointCallbackCount = numChunks; - + if (throttleCheckpointFlushDelayMs >= 0) throttleIndexCheckpointFlushSemaphore = new SemaphoreSlim(0); HashBucket* start = state[version].tableAligned; From 4d17abb72b550b0c4c805957c960332d7af8511f Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 16:48:05 -0800 Subject: [PATCH 17/49] fix garnet --- libs/server/StoreWrapper.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 530d4763d7..bb562ec325 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -668,11 +668,11 @@ private async void IndexAutoGrowTask(CancellationToken token) if (!indexMaxedOut) indexMaxedOut = GrowIndexIfNeeded(StoreType.Main, serverOptions.AdjustedIndexMaxCacheLines, store.OverflowBucketAllocations, - () => store.IndexSize, () => store.GrowIndex()); + () => store.IndexSize, async () => await store.GrowIndexAsync()); if (!objectStoreIndexMaxedOut) objectStoreIndexMaxedOut = GrowIndexIfNeeded(StoreType.Object, serverOptions.AdjustedObjectStoreIndexMaxCacheLines, objectStore.OverflowBucketAllocations, - () => objectStore.IndexSize, () => objectStore.GrowIndex()); + () => objectStore.IndexSize, async () => await objectStore.GrowIndexAsync()); } } catch (Exception ex) From 2b830b1d5fdc72089bb32b8ef846565c2b943bec Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 17:23:55 -0800 Subject: [PATCH 18/49] nit --- .../storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index ad1175a454..9d1c5d5d92 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -7,7 +7,6 @@ using System.Threading; using System.Threading.Tasks; using Microsoft.Extensions.Logging; -using static System.Collections.Specialized.BitVector32; namespace Tsavorite.core { From d92360aed711ebf3186e8c0b1e0f43cc5f57829d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 17:33:54 -0800 Subject: [PATCH 19/49] comments --- .../cs/src/core/Index/Checkpointing/IStateMachine.cs | 8 ++++++++ .../cs/src/core/Index/Checkpointing/IStateMachineTask.cs | 3 +++ .../cs/src/core/Index/Checkpointing/StateMachineDriver.cs | 7 +++++-- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs index fa09556125..f3e43059fb 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachine.cs @@ -3,8 +3,16 @@ namespace Tsavorite.core { + /// + /// State machine API + /// public interface IStateMachine : IStateMachineTask { + /// + /// Returns the next state given the current state + /// + /// Current state + /// Next state public SystemState NextState(SystemState currentState); } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs index 0c73707770..99c899299b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IStateMachineTask.cs @@ -3,6 +3,9 @@ namespace Tsavorite.core { + /// + /// Interface for tasks that are executed as part of the state machine + /// public interface IStateMachineTask { /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 62aa1d9fd1..6653b0f075 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -10,6 +10,9 @@ namespace Tsavorite.core { + /// + /// Driver for the state machine. This class is responsible for executing the state machine. + /// public class StateMachineDriver { SystemState systemState; @@ -205,11 +208,11 @@ async Task RunStateMachine(CancellationToken token = default) _ = Interlocked.Exchange(ref stateMachine, null); if (ex != null) { - _stateMachineCompleted.TrySetException(ex); + _ = _stateMachineCompleted.TrySetException(ex); } else { - _stateMachineCompleted.TrySetResult(true); + _ = _stateMachineCompleted.TrySetResult(true); } } } From 8145607265d4c377792fe0cd692a6b3297e80308 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Thu, 27 Feb 2025 18:34:10 -0800 Subject: [PATCH 20/49] remove manualLockingActive --- .../core/Index/CheckpointManagement/RecoveryInfo.cs | 10 ++++------ .../src/core/Index/Checkpointing/StateMachineBase.cs | 1 + .../Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs | 1 - .../Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs | 1 - 4 files changed, 5 insertions(+), 8 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/RecoveryInfo.cs b/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/RecoveryInfo.cs index 34dd5874ab..a3aaa95dec 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/RecoveryInfo.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/CheckpointManagement/RecoveryInfo.cs @@ -62,10 +62,9 @@ public struct HybridLogRecoveryInfo public long beginAddress; /// - /// If true, there was at least one ITsavoriteContext implementation active that did manual locking at some point during the checkpoint; - /// these pages must be scanned for lock cleanup. + /// Placeholder to avoid checkpoint format change /// - public bool manualLockingActive; + public bool placeholder; /// /// Object log segment offsets @@ -154,7 +153,7 @@ public void Initialize(StreamReader reader) deltaTailAddress = long.Parse(value); value = reader.ReadLine(); - manualLockingActive = bool.Parse(value); + placeholder = bool.Parse(value); value = reader.ReadLine(); var numSessions = int.Parse(value); @@ -262,7 +261,7 @@ public byte[] ToByteArray() writer.WriteLine(headAddress); writer.WriteLine(beginAddress); writer.WriteLine(deltaTailAddress); - writer.WriteLine(manualLockingActive); + writer.WriteLine(placeholder); writer.WriteLine(checkpointTokenCount); @@ -306,7 +305,6 @@ public readonly void DebugPrint(ILogger logger) logger?.LogInformation("Head Address: {headAddress}", headAddress); logger?.LogInformation("Begin Address: {beginAddress}", beginAddress); logger?.LogInformation("Delta Tail Address: {deltaTailAddress}", deltaTailAddress); - logger?.LogInformation("Manual Locking Active: {manualLockingActive}", manualLockingActive); } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs index f1ba3cbe0d..7b0add4b56 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs @@ -16,6 +16,7 @@ internal abstract class StateMachineBase : IStateMachine /// Construct a new SynchronizationStateMachine with the given tasks. The order of tasks given is the /// order they are executed on each state machine. /// + /// To version /// The ISynchronizationTasks to run on the state machine protected StateMachineBase(long toVersion = -1, params IStateMachineTask[] tasks) { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs index 4f9b42d6c7..27daee2063 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs @@ -89,7 +89,6 @@ internal void InitializeIndexCheckpoint(Guid indexToken) internal void InitializeHybridLogCheckpoint(Guid hybridLogToken, long version) { _hybridLogCheckpoint.Initialize(hybridLogToken, version, checkpointManager); - _hybridLogCheckpoint.info.manualLockingActive = hlogBase.NumActiveLockingSessions > 0; } internal long Compact(ISessionFunctions functions, CompactionFunctions compactionFunctions, long untilAddress, CompactionType compactionType) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 56bc19bdcf..33ccdecc1b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -71,7 +71,6 @@ public partial class TsavoriteKV : Ts internal void IncrementNumLockingSessions() { - _hybridLogCheckpoint.info.manualLockingActive = true; Interlocked.Increment(ref hlogBase.NumActiveLockingSessions); } internal void DecrementNumLockingSessions() => Interlocked.Decrement(ref hlogBase.NumActiveLockingSessions); From 4ba164ccc22beea70ce9b775321de105deb8a1da Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 28 Feb 2025 15:52:42 -0800 Subject: [PATCH 21/49] update the barrier condition and remove checkpoint version switch barrier flag. --- libs/host/GarnetServer.cs | 2 - .../HybridLogCheckpointSMTask.cs | 6 +++ .../Index/Checkpointing/IndexResizeSMTask.cs | 8 ++++ .../core/Index/Common/CheckpointSettings.cs | 5 --- .../src/core/Index/Common/ExecutionContext.cs | 9 ++-- .../cs/src/core/Index/Common/KVSettings.cs | 8 +--- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 2 - .../core/Index/Tsavorite/TsavoriteThread.cs | 41 +++++++------------ .../Tsavorite/cs/test/CancellationTests.cs | 10 ++--- .../Tsavorite/cs/test/ExpirationTests.cs | 22 +++++----- .../cs/test/LockableUnsafeContextTests.cs | 6 +-- 11 files changed, 53 insertions(+), 66 deletions(-) diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index 58ac66979f..3ee889f60f 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -302,7 +302,6 @@ private void CreateMainStore(IClusterFactory clusterFactory, out string checkpoi // Run checkpoint on its own thread to control p99 kvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; - kvSettings.CheckpointVersionSwitchBarrier = opts.EnableCluster; if (opts.EnableCluster) { @@ -330,7 +329,6 @@ private void CreateObjectStore(IClusterFactory clusterFactory, CustomCommandMana // Run checkpoint on its own thread to control p99 objKvSettings.ThrottleCheckpointFlushDelayMs = opts.CheckpointThrottleFlushDelayMs; - objKvSettings.CheckpointVersionSwitchBarrier = opts.EnableCluster; if (opts.EnableCluster) objKvSettings.CheckpointManager = clusterFactory.CreateCheckpointManager( diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index a8a55577bc..ce4e5d264f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -2,6 +2,7 @@ // Licensed under the MIT license. using System; +using System.Threading; using System.Threading.Tasks; namespace Tsavorite.core @@ -42,6 +43,11 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; break; case Phase.IN_PROGRESS: + // Wait for PREPARE threads to finish active transactions and enter barrier + while (store.hlogBase.NumActiveLockingSessions > 0) + { + _ = Thread.Yield(); + } store.CheckpointVersionShift(lastVersion, next.Version); break; case Phase.WAIT_FLUSH: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs index 4cccbd893e..6fd5c90d78 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. +using System.Threading; + namespace Tsavorite.core { /// @@ -25,6 +27,12 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state case Phase.PREPARE_GROW: break; case Phase.IN_PROGRESS_GROW: + // Wait for PREPARE_GROW threads to finish active transactions and enter barrier + while (store.hlogBase.NumActiveLockingSessions > 0) + { + _ = Thread.Yield(); + } + // Set up the transition to new version of HT var numChunks = (int)(store.state[store.resizeInfo.version].size / Constants.kSizeofChunk); if (numChunks == 0) numChunks = 1; // at least one chunk diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/CheckpointSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/CheckpointSettings.cs index ac12e9cb0f..dc4867bec0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/CheckpointSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/CheckpointSettings.cs @@ -50,10 +50,5 @@ internal class CheckpointSettings /// Whether we should throttle the disk IO for checkpoints (one write at a time, wait between each write) and issue IO from separate task (-1 = throttling disabled) /// public int ThrottleCheckpointFlushDelayMs = -1; - - /// - /// Whether we use a barrier to ensure that threads are not in two different checkpoint versions at the same time - /// - public bool CheckpointVersionSwitchBarrier = false; } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index 0e8a8a03e0..2ddb97da52 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -20,8 +20,10 @@ internal sealed class TsavoriteExecutionContext // Control automatic Read copy operations. These flags override flags specified at the TsavoriteKV level, but may be overridden on the individual Read() operations internal ReadCopyOptions ReadCopyOptions; - internal long version; - public Phase phase; + public SystemState SessionState; + internal long version => SessionState.Version; + public Phase phase => SessionState.Phase; + public long totalPending; public readonly Dictionary> ioPendingRequests; public readonly AsyncCountDown pendingReads; @@ -31,8 +33,7 @@ internal sealed class TsavoriteExecutionContext public TsavoriteExecutionContext(int sessionID, string sessionName) { - phase = Phase.REST; - version = 1; + SessionState = SystemState.Make(Phase.REST, 1); this.sessionID = sessionID; this.sessionName = sessionName; readyResponses = new AsyncQueue>(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs index 65eb242adf..43d5eab5a1 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs @@ -114,11 +114,6 @@ public sealed class KVSettings : IDisposable /// public int ThrottleCheckpointFlushDelayMs = -1; - /// - /// Whether we use a barrier to ensure that threads are not in two different checkpoint versions at the same time - /// - public bool CheckpointVersionSwitchBarrier = false; - /// /// Settings for recycling deleted records on the log. /// @@ -235,8 +230,7 @@ internal CheckpointSettings GetCheckpointSettings() CheckpointDir = CheckpointDir, CheckpointManager = CheckpointManager, RemoveOutdated = RemoveOutdatedCheckpoints, - ThrottleCheckpointFlushDelayMs = ThrottleCheckpointFlushDelayMs, - CheckpointVersionSwitchBarrier = CheckpointVersionSwitchBarrier + ThrottleCheckpointFlushDelayMs = ThrottleCheckpointFlushDelayMs }; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 33ccdecc1b..fcd0f5cb87 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -66,7 +66,6 @@ public partial class TsavoriteKV : Ts int maxSessionID; - internal readonly bool CheckpointVersionSwitchBarrier; // version switch barrier internal readonly OverflowBucketLockTable LockTable; internal void IncrementNumLockingSessions() @@ -98,7 +97,6 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun var checkpointSettings = kvSettings.GetCheckpointSettings() ?? new CheckpointSettings(); - CheckpointVersionSwitchBarrier = checkpointSettings.CheckpointVersionSwitchBarrier; ThrottleCheckpointFlushDelayMs = checkpointSettings.ThrottleCheckpointFlushDelayMs; if (checkpointSettings.CheckpointDir != null && checkpointSettings.CheckpointManager != null) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index e03970683e..9f4fed51c5 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -18,40 +18,27 @@ internal void InternalRefresh Date: Fri, 28 Feb 2025 16:19:14 -0800 Subject: [PATCH 22/49] remove INTERMEDIATE state --- .../Index/Checkpointing/StateTransitions.cs | 19 ---------------- .../Tsavorite/cs/test/CancellationTests.cs | 10 ++++----- .../Tsavorite/cs/test/ExpirationTests.cs | 22 +++++++++---------- .../cs/test/LockableUnsafeContextTests.cs | 6 ++--- .../cs/test/StateMachineBarrierTests.cs | 3 +-- 5 files changed, 20 insertions(+), 40 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs index 0c8291db18..0579fc5d63 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs @@ -58,9 +58,6 @@ public enum Phase : int /// Index resizing is in progress IN_PROGRESS_GROW, - - /// Internal intermediate state of state machine - INTERMEDIATE = 16, }; /// @@ -145,22 +142,6 @@ internal static SystemState Make(Phase status, long version) return info; } - /// - /// Create a copy of the passed that is marked with the phase - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static SystemState MakeIntermediate(SystemState state) - => Make(state.Phase | Phase.INTERMEDIATE, state.Version); - - /// - /// Create a copy of the passed that is not marked with the phase - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal static void RemoveIntermediate(ref SystemState state) - { - state.Phase &= ~Phase.INTERMEDIATE; - } - /// /// Compare two s for equality /// diff --git a/libs/storage/Tsavorite/cs/test/CancellationTests.cs b/libs/storage/Tsavorite/cs/test/CancellationTests.cs index fbd685c59b..9d85f899c9 100644 --- a/libs/storage/Tsavorite/cs/test/CancellationTests.cs +++ b/libs/storage/Tsavorite/cs/test/CancellationTests.cs @@ -171,7 +171,7 @@ private unsafe void Populate() [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void InitialUpdaterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void InitialUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); @@ -191,7 +191,7 @@ public void InitialUpdaterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase ph [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void CopyUpdaterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void CopyUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); @@ -224,7 +224,7 @@ void do_it() [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void InPlaceUpdaterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void InPlaceUpdaterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); @@ -240,7 +240,7 @@ public void InPlaceUpdaterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase ph [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SingleWriterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void SingleWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); @@ -255,7 +255,7 @@ public void SingleWriterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phas [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void ConcurrentWriterTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void ConcurrentWriterTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { Populate(); session.ctx.SessionState = SystemState.Make(phase, session.ctx.version); diff --git a/libs/storage/Tsavorite/cs/test/ExpirationTests.cs b/libs/storage/Tsavorite/cs/test/ExpirationTests.cs index 2e2c5384b8..8bfa751e9f 100644 --- a/libs/storage/Tsavorite/cs/test/ExpirationTests.cs +++ b/libs/storage/Tsavorite/cs/test/ExpirationTests.cs @@ -701,7 +701,7 @@ private static Funcs GetExpectedFuncs(FlushMode flushMode, Funcs noFlush, Funcs [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void PassiveExpireTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void PassiveExpireTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -713,7 +713,7 @@ public void PassiveExpireTest([Values] FlushMode flushMode, [Values(Phase.REST, [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void ExpireDeleteTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void ExpireDeleteTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -743,7 +743,7 @@ public void ExpireDeleteTest([Values] FlushMode flushMode, [Values(Phase.REST, P [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void ExpireRolloverTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void ExpireRolloverTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -772,7 +772,7 @@ public void ExpireRolloverTest([Values] FlushMode flushMode, [Values(Phase.REST, [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SetIfKeyExistsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void SetIfKeyExistsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -816,7 +816,7 @@ public void SetIfKeyExistsTest([Values] FlushMode flushMode, [Values(Phase.REST, [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SetIfKeyNotExistsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void SetIfKeyNotExistsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -861,7 +861,7 @@ public void SetIfKeyNotExistsTest([Values] FlushMode flushMode, [Values(Phase.RE [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SetIfValueEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void SetIfValueEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -907,7 +907,7 @@ public void SetIfValueEqualsTest([Values] FlushMode flushMode, [Values(Phase.RES [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void SetIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void SetIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -950,7 +950,7 @@ public void SetIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Phase. [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void DeleteThenUpdateTest([Values] FlushMode flushMode, [Values] KeyEquality keyEquality, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void DeleteThenUpdateTest([Values] FlushMode flushMode, [Values] KeyEquality keyEquality, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -999,7 +999,7 @@ public void DeleteThenUpdateTest([Values] FlushMode flushMode, [Values] KeyEqual [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void DeleteThenInsertTest([Values] FlushMode flushMode, [Values] KeyEquality keyEquality, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void DeleteThenInsertTest([Values] FlushMode flushMode, [Values] KeyEquality keyEquality, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); @@ -1047,7 +1047,7 @@ public void DeleteThenInsertTest([Values] FlushMode flushMode, [Values] KeyEqual [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void DeleteAndCancelIfValueEqualsTest([Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void DeleteAndCancelIfValueEqualsTest([Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); const TestOp testOp = TestOp.DeleteIfValueEqualsAndStop; @@ -1083,7 +1083,7 @@ public void DeleteAndCancelIfValueEqualsTest([Values(Phase.REST, Phase.INTERMEDI [Test] [Category("TsavoriteKV")] [Category("Smoke"), Category("RMW")] - public void DeleteIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + public void DeleteIfValueNotEqualsTest([Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) { InitialIncrement(); MaybeEvict(flushMode); diff --git a/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs b/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs index c6478a6208..822a4a515b 100644 --- a/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs +++ b/libs/storage/Tsavorite/cs/test/LockableUnsafeContextTests.cs @@ -471,7 +471,7 @@ public async Task TestShiftHeadAddressLUC([Values] CompletionSyncMode syncMode) [Category(LockableUnsafeContextTestCategory)] [Category(SmokeTestCategory)] public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget, - [Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase, + [Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase, [Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp updateOp) { Populate(); @@ -610,7 +610,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget [Test] [Category(LockableUnsafeContextTestCategory)] [Category(SmokeTestCategory)] - public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Values] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase, + public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Values] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase, [Values(UpdateOp.Upsert, UpdateOp.RMW)] UpdateOp updateOp) { Populate(); @@ -741,7 +741,7 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va [Category(SmokeTestCategory)] #pragma warning disable IDE0060 // Remove unused parameter: readCopyDestination is used by Setup public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Values] ReadCopyDestination readCopyDestination, - [Values(FlushMode.NoFlush, FlushMode.ReadOnly)] FlushMode flushMode, [Values(Phase.REST, Phase.INTERMEDIATE)] Phase phase) + [Values(FlushMode.NoFlush, FlushMode.ReadOnly)] FlushMode flushMode, [Values(Phase.REST, Phase.PREPARE)] Phase phase) #pragma warning restore IDE0060 // Remove unused parameter { // Phase.INTERMEDIATE is to test the non-Phase.REST blocks diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index 6963ee37b4..a453ac7b77 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -39,8 +39,7 @@ public void Setup() MutableFraction = 0.1, PageSize = 1L << 10, MemorySize = 1L << 13, - CheckpointDir = checkpointDir, - CheckpointVersionSwitchBarrier = true + CheckpointDir = checkpointDir }, StoreFunctions.Create(new AdId.Comparer()) , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); From 528c2272f632ae09b2abbb9786351e61e07a0227 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 28 Feb 2025 18:29:49 -0800 Subject: [PATCH 23/49] Remove CPR_SHIFT_DETECTED and LartchDestination.Retry --- .../src/core/Index/Common/OperationStatus.cs | 5 ----- .../Implementation/ContinuePending.cs | 4 ---- .../Implementation/HandleOperationStatus.cs | 5 ----- .../Index/Tsavorite/Implementation/Helpers.cs | 18 ++------------- .../Implementation/InternalDelete.cs | 6 ----- .../Tsavorite/Implementation/InternalRMW.cs | 22 +++---------------- .../Tsavorite/Implementation/InternalRead.cs | 12 ---------- .../Implementation/InternalUpsert.cs | 12 +--------- 8 files changed, 6 insertions(+), 78 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs index 5a15ea78d6..c89e7e096b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/OperationStatus.cs @@ -64,11 +64,6 @@ internal enum OperationStatus /// RECORD_ON_DISK, - /// - /// A checkpoint is in progress so the operation must be retried internally after refreshing the epoch and updating the session context version. - /// - CPR_SHIFT_DETECTED, - /// /// Allocation failed, due to a need to flush pages. Clients do not see this status directly; they see . /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs index b6c479f673..7b9ca61e61 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/ContinuePending.cs @@ -65,10 +65,6 @@ internal OperationStatus ContinuePendingRead(sessionFunctions); Thread.Yield(); return true; - case OperationStatus.CPR_SHIFT_DETECTED: - // Retry as (v+1) Operation - SynchronizeEpoch(sessionFunctions.Ctx, ref pendingContext, sessionFunctions); - return true; case OperationStatus.ALLOCATE_FAILED: // Async handles this in its own way, as part of the *AsyncResult.Complete*() sequence. Debug.Assert(!pendingContext.flushEvent.IsDefault(), "flushEvent is required for ALLOCATE_FAILED"); @@ -119,7 +115,6 @@ internal Status HandleOperationStatus( { Debug.Assert(operationStatus != OperationStatus.RETRY_NOW, "OperationStatus.RETRY_NOW should have been handled before HandleOperationStatus"); Debug.Assert(operationStatus != OperationStatus.RETRY_LATER, "OperationStatus.RETRY_LATER should have been handled before HandleOperationStatus"); - Debug.Assert(operationStatus != OperationStatus.CPR_SHIFT_DETECTED, "OperationStatus.CPR_SHIFT_DETECTED should have been handled before HandleOperationStatus"); request = default; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs index cee1505547..21fc63641f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/Helpers.cs @@ -14,8 +14,7 @@ public unsafe partial class TsavoriteKV(long logicalAddress, Tsavorite } /// - /// This is a wrapper for checking the record's version instead of just peeking at the latest record at the tail of the bucket. - /// By calling with the address of the traced record, we can prevent a different key sharing the same bucket from deceiving - /// the operation to think that the version of the key has reached v+1 and thus to incorrectly update in place. + /// Check the version of the passed-in record. /// /// The logical address of the traced record for the key /// @@ -47,18 +44,7 @@ internal void MarkPage(long logicalAddress, Tsavorite private bool IsRecordVersionNew(long logicalAddress) { HashBucketEntry entry = new() { word = logicalAddress }; - return IsEntryVersionNew(ref entry); - } - /// - /// Check the version of the passed-in entry. - /// The semantics of this function are to check the tail of a bucket (indicated by entry), so we name it this way. - /// - /// the last entry of a bucket - /// - [MethodImpl(MethodImplOptions.AggressiveInlining)] - private bool IsEntryVersionNew(ref HashBucketEntry entry) - { // A version shift can only happen in an address after the checkpoint starts, as v_new threads RCU entries to the tail. if (entry.Address < _hybridLogCheckpoint.info.startLogicalAddress) return false; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs index be3c3a7cca..fae488e69a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalDelete.cs @@ -33,10 +33,6 @@ public unsafe partial class TsavoriteKVRETRY_LATER /// Cannot be processed immediately due to system state. Add to pending list and retry later /// - /// - /// CPR_SHIFT_DETECTED - /// A shift in version has been detected. Synchronize immediately to avoid violating CPR consistency. - /// /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -88,8 +84,6 @@ internal OperationStatus InternalDeleteRETRY_LATER /// Cannot be processed immediately due to system state. Add to pending list and retry later. /// - /// - /// CPR_SHIFT_DETECTED - /// A shift in version has been detected. Synchronize immediately to avoid violating CPR consistency. - /// /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -89,8 +85,6 @@ internal OperationStatus InternalRMW= hlogBase.HeadAddress) srcRecordInfo = ref stackCtx.recSrc.GetInfo(); @@ -301,23 +295,13 @@ private bool TryRevivifyInChain stackCtx, ref OperationStatus status, ref LatchOperation latchOperation) { - // The idea of CPR is that if a thread in version V tries to perform an operation and notices a record in V+1, it needs to back off and run CPR_SHIFT_DETECTED. - // Similarly, a V+1 thread cannot update a V record; it needs to do a read-copy-update (or upsert at tail) instead of an in-place update. - // For background info: Prior to HashBucket-based locking, we had to lock the bucket in the following way: - // 1. V threads take shared lock on bucket - // 2. V+1 threads take exclusive lock on bucket, refreshing until they can - // 3. If V thread cannot take shared lock, that means the system is in V+1 so we can immediately refresh and go to V+1 (do CPR_SHIFT_DETECTED) - // 4. If V thread manages to get shared lock, but encounters a V+1 record, it knows the system is in V+1 so it will do CPR_SHIFT_DETECTED - // Now we no longer need to do the bucket latching, since we already have a latch on the bucket. + // With version switch barrier, a thread in version V will never notice a record in V+1. + // A V+1 thread cannot update a V record; it needs to do a read-copy-update (or upsert at tail) instead of an in-place update. switch (phase) { case Phase.PREPARE: // Thread is in V - if (!IsEntryVersionNew(ref stackCtx.hei.entry)) - break; // Normal Processing; thread is in V, record is in V - - status = OperationStatus.CPR_SHIFT_DETECTED; - return LatchDestination.Retry; // Pivot Thread for retry (do not operate on v+1 record when thread is in V) + break; case Phase.IN_PROGRESS: // Thread is in v+1 case Phase.WAIT_INDEX_CHECKPOINT: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRead.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRead.cs index e20c675b98..bf97843287 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRead.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRead.cs @@ -36,10 +36,6 @@ public unsafe partial class TsavoriteKVThe record corresponding to 'key' is on disk and the operation. /// /// - /// CPR_SHIFT_DETECTED - /// A shift in version has been detected. Synchronize immediately to avoid violating CPR consistency. - /// - /// /// RETRY_LATER /// Refresh the epoch and retry. /// @@ -111,10 +107,6 @@ internal OperationStatus InternalRead= hlogBase.SafeReadOnlyAddress) { // Mutable region (even fuzzy region is included here) @@ -232,10 +224,6 @@ private static OperationStatus CheckFalseActionStatus(ReadInfo readInfo) /// The record corresponding to 'key' is on disk and the operation. /// /// - /// CPR_SHIFT_DETECTED - /// A shift in version has been detected. Synchronize immediately to avoid violating CPR consistency. - /// - /// /// RETRY_LATER /// Refresh the epoch and retry. /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs index f87e5b0303..bd045e8037 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs @@ -36,10 +36,6 @@ public unsafe partial class TsavoriteKVRETRY_LATER /// Cannot be processed immediately due to system state. Add to pending list and retry later /// - /// - /// CPR_SHIFT_DETECTED - /// A shift in version has been detected. Synchronize immediately to avoid violating CPR consistency. - /// /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -83,8 +79,6 @@ internal OperationStatus InternalUpsert= hlogBase.HeadAddress) srcRecordInfo = ref stackCtx.recSrc.GetInfo(); @@ -272,11 +266,7 @@ private LatchDestination CheckCPRConsistencyUpsert(Phase phase, ref OperationSta switch (phase) { case Phase.PREPARE: // Thread is in V - if (!IsEntryVersionNew(ref stackCtx.hei.entry)) - break; // Normal Processing; thread is in V, record is in V - - status = OperationStatus.CPR_SHIFT_DETECTED; - return LatchDestination.Retry; // Pivot Thread for retry (do not operate on V+1 record when thread is in V) + break; case Phase.IN_PROGRESS: // Thread is in V+1 case Phase.WAIT_INDEX_CHECKPOINT: From 2067cb942bbcb53a5737ec394f69acf3000c2c2f Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 11:04:09 -0800 Subject: [PATCH 24/49] add black box test for checkpointing version switch state machine remove deprecated white box test --- .../cs/test/StateMachineBarrierTests.cs | 9 + .../cs/test/StateMachineDriverTests.cs | 194 ++++++ .../Tsavorite/cs/test/StateMachineTests.cs | 642 ------------------ 3 files changed, 203 insertions(+), 642 deletions(-) create mode 100644 libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs delete mode 100644 libs/storage/Tsavorite/cs/test/StateMachineTests.cs diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index a453ac7b77..9bb60e1d12 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -151,4 +151,13 @@ void Prepare(out SimpleFunctions f, ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); } } + + public class SimpleFunctions : SimpleSessionFunctions + { + public override void ReadCompletionCallback(ref AdId key, ref NumClicks input, ref NumClicks output, Empty ctx, Status status, RecordMetadata recordMetadata) + { + ClassicAssert.IsTrue(status.Found); + ClassicAssert.AreEqual(key.adId, output.numClicks); + } + } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs new file mode 100644 index 0000000000..68e385a165 --- /dev/null +++ b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs @@ -0,0 +1,194 @@ +// Copyright (c) Microsoft Corporation. +// Licensed under the MIT license. + +using System; +using System.IO; +using System.Threading; +using System.Threading.Tasks; +using NUnit.Framework; +using NUnit.Framework.Legacy; +using Tsavorite.core; + +namespace Tsavorite.test.recovery +{ + using static Tsavorite.test.TestUtils; + using LongAllocator = BlittableAllocator>>; + using LongStoreFunctions = StoreFunctions>; + + class SumFunctions : SimpleSimpleFunctions + { + public SumFunctions() : base((l, r) => l + r) { } + } + + [TestFixture] + public class CheckpointVersionSwitchTest + { + IDevice log; + bool rmwDone; + long[] expectedV1Count; + long[] expectedV2Count; + readonly int numKeys = 2; + + [SetUp] + public void Setup() + { + rmwDone = false; + expectedV1Count = new long[numKeys]; + expectedV2Count = new long[numKeys]; + log = CreateTestDevice(DeviceType.LSD, Path.Join(MethodTestDir, "Test.log")); + } + + [TearDown] + public void TearDown() + { + log?.Dispose(); + log = null; + DeleteDirectory(MethodTestDir, true); + } + + void RmwOperationThread(int thread_id, TsavoriteKV store) + { + using var s = store.NewSession(new SumFunctions()); + var bc = s.BasicContext; + var r = new Random(thread_id); + + long key = 0; + long input = 1; + var v1count = new long[numKeys]; + var v2count = new long[numKeys]; + while (!rmwDone) + { + key = r.Next(numKeys); + _ = bc.RMW(ref key, ref input); + if (bc.Session.Version == 1) + v1count[key]++; + v2count[key]++; + } + for (int i = 0; i < numKeys; i++) + { + _ = Interlocked.Add(ref expectedV1Count[i], v1count[i]); + _ = Interlocked.Add(ref expectedV2Count[i], v2count[i]); + } + } + + [Test] + public async ValueTask CheckpointVersionSwitchTest1( + [Values(CheckpointType.Snapshot, CheckpointType.FoldOver)] CheckpointType checkpointType, + [Values] bool isAsync, [Values(1L << 13, 1L << 16)] long indexSize) + { + // Create the original store + using var store1 = new TsavoriteKV(new() + { + IndexSize = indexSize, + LogDevice = log, + PageSize = 1L << 10, + MemorySize = 1L << 20, + CheckpointDir = MethodTestDir + }, StoreFunctions.Create(LongKeyComparer.Instance) + , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) + ); + + // Start RMW operation threads + int NumRmwThreads = 2; + var rmwTasks = new Task[NumRmwThreads]; + for (int i = 0; i < NumRmwThreads; i++) + { + var thread_id = i; + rmwTasks[i] = Task.Run(() => RmwOperationThread(thread_id, store1)); + } + + // Wait for some RMWs to complete in v1 + await Task.Delay(500); + + // Initiate checkpoint concurrent to the RMW operation threads + var task = store1.TakeFullCheckpointAsync(checkpointType); + + // Wait for the checkpoint to complete + Guid token; + if (isAsync) + { + (var status, token) = await task; + } + else + { + (var status, token) = task.AsTask().GetAwaiter().GetResult(); + } + + // Wait for some RMWs to complete in v2 + await Task.Delay(500); + + // Signal RMW threads to stop + rmwDone = true; + await Task.WhenAll(rmwTasks); + + // Verify the final state of the old store + using var s1 = store1.NewSession(new SumFunctions()); + var bc1 = s1.BasicContext; + for (long key = 0; key < numKeys; key++) + { + long output = default; + var status = bc1.Read(ref key, ref output); + if (status.IsPending) + { + var completed = bc1.CompletePendingWithOutputs(out var completedOutputs, true); + ClassicAssert.IsTrue(completed); + bool result = completedOutputs.Next(); + ClassicAssert.IsTrue(result); + status = completedOutputs.Current.Status; + output = completedOutputs.Current.Output; + result = completedOutputs.Next(); + ClassicAssert.IsFalse(result); + } + ClassicAssert.IsTrue(status.Found, $"status = {status}"); + + // The old store should have the latest state + ClassicAssert.AreEqual(expectedV2Count[key], output, $"output = {output}"); + } + + // Recover new store from the checkpoint + using var store2 = new TsavoriteKV(new() + { + IndexSize = indexSize, + LogDevice = log, + MutableFraction = 1, + PageSize = 1L << 10, + MemorySize = 1L << 20, + CheckpointDir = MethodTestDir + }, StoreFunctions.Create(LongKeyComparer.Instance) + , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions)); + + if (isAsync) + { + _ = await store2.RecoverAsync(default, token); + } + else + { + _ = store2.Recover(default, token); + } + + // Verify the state of the new store + using var s2 = store2.NewSession(new SumFunctions()); + var bc2 = s2.BasicContext; + for (long key = 0; key < numKeys; key++) + { + long output = default; + var status = bc2.Read(ref key, ref output); + if (status.IsPending) + { + var completed = bc2.CompletePendingWithOutputs(out var completedOutputs, true); + ClassicAssert.IsTrue(completed); + bool result = completedOutputs.Next(); + ClassicAssert.IsTrue(result); + status = completedOutputs.Current.Status; + output = completedOutputs.Current.Output; + result = completedOutputs.Next(); + ClassicAssert.IsFalse(result); + } + ClassicAssert.IsTrue(status.Found, $"status = {status}"); + + // The new store should have state as of V1, and not the latest state of the old store + ClassicAssert.AreEqual(expectedV1Count[key], output, $"output = {output}"); + } + } + } +} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineTests.cs deleted file mode 100644 index 3dc3873a92..0000000000 --- a/libs/storage/Tsavorite/cs/test/StateMachineTests.cs +++ /dev/null @@ -1,642 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System.Collections.Generic; -using System.IO; -using System.Threading; -using NUnit.Framework; -using NUnit.Framework.Legacy; -using Tsavorite.core; -using Tsavorite.test.recovery.sumstore; - -namespace Tsavorite.test.statemachine -{ - using StructAllocator = BlittableAllocator>>; - using StructStoreFunctions = StoreFunctions>; - - [TestFixture] - [Ignore("State machine has changed")] - public class StateMachineTests - { - IDevice log; - TsavoriteKV store; - const int NumOps = 5000; - AdId[] inputArray; - - [SetUp] - public void Setup() - { - inputArray = new AdId[NumOps]; - for (int i = 0; i < NumOps; i++) - inputArray[i].adId = i; - - log = Devices.CreateLogDevice(Path.Join(TestUtils.MethodTestDir, "StateMachineTest1.log"), deleteOnClose: true); - string checkpointDir = Path.Join(TestUtils.MethodTestDir, "statemachinetest"); - _ = Directory.CreateDirectory(checkpointDir); - - store = new(new() - { - IndexSize = 1L << 13, - LogDevice = log, - MutableFraction = 0.1, - PageSize = 1L << 10, - MemorySize = 1L << 13, - CheckpointDir = checkpointDir - }, StoreFunctions.Create(new AdId.Comparer()) - , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) - ); - } - - [TearDown] - public void TearDown() - { - store?.Dispose(); - store = null; - log?.Dispose(); - log = null; - TestUtils.DeleteDirectory(TestUtils.MethodTestDir); - } - - [TestCase] - [Category("TsavoriteKV")] - [Category("CheckpointRestore")] - [Category("Smoke")] - public void StateMachineTest1() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s2 - s2.Refresh(); - - // s1 has not refreshed, so we should still be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh s1 - uc1.Refresh(); - - // We should now be in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - - s2.Refresh(); - - // We should be in WAIT_FLUSH, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.WAIT_FLUSH, 2), store.SystemState)); - - uc1.Refresh(); - - // We should be in PERSISTENCE_CALLBACK, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PERSISTENCE_CALLBACK, 2), store.SystemState)); - - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - // Dispose session s2; does not move state machine forward - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void StateMachineTest2() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s2 - s2.Refresh(); - - // s1 has not refreshed, so we should still be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh s1 - uc1.Refresh(); - - // We should now be in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - - // Dispose session s2; moves state machine forward to WAIT_FLUSH, 2 - s2.Dispose(); - - // We should be in WAIT_FLUSH, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.WAIT_FLUSH, 2), store.SystemState)); - - // Since s1 is the only session now, it will fast-foward state machine - // to completion - uc1.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void StateMachineTest3() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s1 - uc1.Refresh(); - - // s1 is now in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), SystemState.Make(s1.ctx.phase, s1.ctx.version))); - - // Suspend s1 - uc1.EndUnsafe(); - - // Since s2 is the only session now, it will fast-foward state machine - // to completion - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - uc1.BeginUnsafe(); - - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void StateMachineTest4() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s2 - s2.Refresh(); - - // s1 has not refreshed, so we should still be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh s1 - uc1.Refresh(); - - // We should now be in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - - // s1 is now in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), SystemState.Make(s1.ctx.phase, s1.ctx.version))); - - // Suspend s1 - uc1.EndUnsafe(); - - // Since s2 is the only session now, it will fast-foward state machine - // to completion - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - uc1.BeginUnsafe(); - - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void StateMachineTest5() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s2 - uc1.Refresh(); - s2.Refresh(); - - // We should now be in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - - uc1.Refresh(); - - // We should be in WAIT_FLUSH, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.WAIT_FLUSH, 2), store.SystemState)); - - - s2.Refresh(); - - // We should be in PERSISTENCE_CALLBACK, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PERSISTENCE_CALLBACK, 2), store.SystemState)); - - uc1.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - // No callback here since already done - uc1.Refresh(); - - // Suspend s1 - uc1.EndUnsafe(); - - // Since s2 is the only session now, it will fast-foward state machine - // to completion - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - uc1.BeginUnsafe(); - - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void StateMachineTest6() - { - Prepare(out _, out var s1, out var uc1, out var s2); - - // Suspend s1 - uc1.EndUnsafe(); - - // s1 is now in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), SystemState.Make(s1.ctx.phase, s1.ctx.version))); - - // System should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Since s2 is the only session now, it will fast-foward state machine - // to completion - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - s2.Dispose(); - - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver); - store.CompleteCheckpointAsync().AsTask().GetAwaiter().GetResult(); - - // We should be in REST, 3 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 3), store.SystemState)); - - uc1.BeginUnsafe(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void LUCScenario1() - { - CreateSessions(out _, out var s1, out var ts, out var lts); - // System should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - lts.getLUC(); - ClassicAssert.IsTrue(lts.isProtected); - - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver); - - // System should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - ts.Refresh(); - lts.Refresh(); - - // System should be in PREPARE, 1 Since there is an active locking session - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - lts.DisposeLUC(); - - ts.Refresh(); - // fast-foward state machine to completion - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - lts.Refresh(); - - s1.Dispose(); - ts.Dispose(); - lts.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void LUCScenario2() - { - CreateSessions(out _, out var s1, out var ts, out var lts); - - // System should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - var uc1 = s1.UnsafeContext; - uc1.BeginUnsafe(); - - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver); - - // should not succeed since checkpoint is in progress - lts.getLUC(); - ClassicAssert.IsFalse(lts.isProtected); - - // We should be in PREPARE phase - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - ts.Refresh(); - // System should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // should not succeed since checkpoint is in progress - lts.getLUC(); - ClassicAssert.IsFalse(lts.isProtected); - - uc1.EndUnsafe(); - - // fast-foward state machine to completion - ts.Refresh(); - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - // should be true since checkpoint is done - lts.getLUC(); - ClassicAssert.IsTrue(lts.isProtected); - lts.DisposeLUC(); - - s1.Dispose(); - ts.Dispose(); - lts.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV"), Category("CheckpointRestore")] - public void LUCScenario3() - { - CreateSessions(out _, out var s1, out var ts, out var lts); - - // System should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - // Start first LUC before checkpoint - var luc1 = s1.LockableUnsafeContext; - luc1.BeginUnsafe(); - luc1.BeginLockable(); - - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver); - - // System should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - luc1.Refresh(); - ts.Refresh(); - luc1.Refresh(); - - // System should be in PREPARE, 1 Since there is an active locking session - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // should not let new LUC start since checkpoint is in progress - lts.getLUC(); - ClassicAssert.IsFalse(lts.isProtected); - - // We still should be in PREPARE phase - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // End first LUC - luc1.EndLockable(); - luc1.EndUnsafe(); - - s1.BasicContext.Refresh(); - // System should be in IN_PROGRESS, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - - // should be true since checkpoint is in IN_PROGRESS phase - lts.getLUC(); - ClassicAssert.IsTrue(lts.isProtected); - lts.DisposeLUC(); - - ts.Refresh(); - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - - // Expect checkpoint completion callback - s1.Dispose(); - ts.Dispose(); - lts.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV")] - [Category("CheckpointRestore")] - [Category("Smoke")] - public void StateMachineCallbackTest1() - { - var callback = new TestCallback(); - store.stateMachineDriver.UnsafeRegisterCallback(callback); - Prepare(out _, out var s1, out var uc1, out var s2); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - callback.CheckInvoked(store.SystemState); - - // Refresh session s2 - s2.Refresh(); - uc1.Refresh(); - - // We should now be in IN_PROGRESS, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, 2), store.SystemState)); - callback.CheckInvoked(store.SystemState); - - s2.Refresh(); - - // We should be in WAIT_FLUSH, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.WAIT_FLUSH, 2), store.SystemState)); - callback.CheckInvoked(store.SystemState); - - uc1.Refresh(); - - // We should be in PERSISTENCE_CALLBACK, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PERSISTENCE_CALLBACK, 2), store.SystemState)); - callback.CheckInvoked(store.SystemState); - - s2.Refresh(); - - // We should be in REST, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 2), store.SystemState)); - callback.CheckInvoked(store.SystemState); - - // Dispose session s2; does not move state machine forward - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - [TestCase] - [Category("TsavoriteKV")] - [Category("CheckpointRestore")] - public void VersionChangeTest() - { - var toVersion = 1 + (1 << 14); - Prepare(out _, out var s1, out var uc1, out var s2, toVersion); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - - // Refresh session s2 - s2.Refresh(); - uc1.Refresh(); - - // We should now be in IN_PROGRESS, toVersion - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.IN_PROGRESS, toVersion), store.SystemState)); - - s2.Refresh(); - - // We should be in WAIT_FLUSH, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.WAIT_FLUSH, toVersion), store.SystemState)); - - uc1.Refresh(); - - // We should be in PERSISTENCE_CALLBACK, 2 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PERSISTENCE_CALLBACK, toVersion), store.SystemState)); - - s2.Refresh(); - - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, toVersion), store.SystemState)); - - - // Dispose session s2; does not move state machine forward - s2.Dispose(); - - uc1.EndUnsafe(); - s1.Dispose(); - } - - void Prepare(out SimpleFunctions f, - out ClientSession s1, - out UnsafeContext uc1, - out ThreadSession s2, - long toVersion = -1) - { - f = new SimpleFunctions(); - - // We should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - // Take index checkpoint for recovery purposes - _ = store.TryInitiateIndexCheckpoint(out _); - store.CompleteCheckpointAsync().AsTask().GetAwaiter().GetResult(); - - // Index checkpoint does not update version, so - // we should still be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - NumClicks value; - - s1 = store.NewSession(f, "foo"); - var bc1 = s1.BasicContext; - - for (int key = 0; key < NumOps; key++) - { - value.numClicks = key; - _ = bc1.Upsert(ref inputArray[key], ref value, Empty.Default); - } - - // Ensure state machine needs no I/O wait during WAIT_FLUSH - store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, true); - - // Create unsafe context and hold epoch to prepare for manual state machine driver - uc1 = s1.UnsafeContext; - uc1.BeginUnsafe(); - - // Start session s2 on another thread for testing - s2 = store.CreateThreadSession(f); - - // We should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver, targetVersion: toVersion); - - // Wait for PREPARE phase - while (!SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)) - Thread.Yield(); - - // We should be in PREPARE, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState)); - } - - - void CreateSessions(out SimpleFunctions f, - out ClientSession s1, - out ThreadSession ts, - out LUCThreadSession lts) - { - f = new SimpleFunctions(); - NumClicks value; - - s1 = store.NewSession(f, "foo"); - var bc1 = s1.BasicContext; - - for (int key = 0; key < NumOps; key++) - { - value.numClicks = key; - _ = bc1.Upsert(ref inputArray[key], ref value, Empty.Default); - } - - // Ensure state machine needs no I/O wait during WAIT_FLUSH - store.Log.ShiftReadOnlyAddress(store.Log.TailAddress, true); - - // Start session s2 on another thread for testing - ts = store.CreateThreadSession(f); - lts = store.CreateLUCThreadSession(f); - - // We should be in REST, 1 - ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - - // Take index checkpoint for recovery purposes - _ = store.TryInitiateIndexCheckpoint(out _); - store.CompleteCheckpointAsync().AsTask().GetAwaiter().GetResult(); - } - } - - public class SimpleFunctions : SimpleSessionFunctions - { - public override void ReadCompletionCallback(ref AdId key, ref NumClicks input, ref NumClicks output, Empty ctx, Status status, RecordMetadata recordMetadata) - { - ClassicAssert.IsTrue(status.Found); - ClassicAssert.AreEqual(key.adId, output.numClicks); - } - } - - public class TestCallback : IStateMachineCallback - { - private readonly HashSet invokedStates = []; - - public void BeforeEnteringState(SystemState next) - { - ClassicAssert.IsFalse(invokedStates.Contains(next)); - _ = invokedStates.Add(next); - } - - public void CheckInvoked(SystemState state) - { - ClassicAssert.IsTrue(invokedStates.Contains(state)); - } - } -} \ No newline at end of file From 174ae9a520dd66a92c9e839aa69283782b8a6078 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 13:27:15 -0800 Subject: [PATCH 25/49] add transaction test --- .../cs/test/StateMachineDriverTests.cs | 208 +++++++++++++----- 1 file changed, 155 insertions(+), 53 deletions(-) diff --git a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs index 68e385a165..2bebc718be 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs @@ -15,64 +15,34 @@ namespace Tsavorite.test.recovery using LongAllocator = BlittableAllocator>>; using LongStoreFunctions = StoreFunctions>; - class SumFunctions : SimpleSimpleFunctions - { - public SumFunctions() : base((l, r) => l + r) { } - } - - [TestFixture] - public class CheckpointVersionSwitchTest + public abstract class StateMachineDriverTestsBase { + readonly int numOpThreads = 2; IDevice log; - bool rmwDone; - long[] expectedV1Count; - long[] expectedV2Count; - readonly int numKeys = 2; - [SetUp] - public void Setup() + protected bool opsDone; + protected long[] expectedV1Count; + protected long[] expectedV2Count; + protected readonly int numKeys = 2; + + protected void BaseSetup() { - rmwDone = false; + opsDone = false; expectedV1Count = new long[numKeys]; expectedV2Count = new long[numKeys]; log = CreateTestDevice(DeviceType.LSD, Path.Join(MethodTestDir, "Test.log")); } - [TearDown] - public void TearDown() + protected void BaseTearDown() { log?.Dispose(); log = null; DeleteDirectory(MethodTestDir, true); } - void RmwOperationThread(int thread_id, TsavoriteKV store) - { - using var s = store.NewSession(new SumFunctions()); - var bc = s.BasicContext; - var r = new Random(thread_id); - - long key = 0; - long input = 1; - var v1count = new long[numKeys]; - var v2count = new long[numKeys]; - while (!rmwDone) - { - key = r.Next(numKeys); - _ = bc.RMW(ref key, ref input); - if (bc.Session.Version == 1) - v1count[key]++; - v2count[key]++; - } - for (int i = 0; i < numKeys; i++) - { - _ = Interlocked.Add(ref expectedV1Count[i], v1count[i]); - _ = Interlocked.Add(ref expectedV2Count[i], v2count[i]); - } - } + protected abstract void OperationThread(int thread_id, TsavoriteKV store); - [Test] - public async ValueTask CheckpointVersionSwitchTest1( + public async ValueTask DoCheckpointVersionSwitchEquivalenceCheck( [Values(CheckpointType.Snapshot, CheckpointType.FoldOver)] CheckpointType checkpointType, [Values] bool isAsync, [Values(1L << 13, 1L << 16)] long indexSize) { @@ -88,19 +58,18 @@ public async ValueTask CheckpointVersionSwitchTest1( , (allocatorSettings, storeFunctions) => new(allocatorSettings, storeFunctions) ); - // Start RMW operation threads - int NumRmwThreads = 2; - var rmwTasks = new Task[NumRmwThreads]; - for (int i = 0; i < NumRmwThreads; i++) + // Start operation threads + var opTasks = new Task[numOpThreads]; + for (int i = 0; i < numOpThreads; i++) { var thread_id = i; - rmwTasks[i] = Task.Run(() => RmwOperationThread(thread_id, store1)); + opTasks[i] = Task.Run(() => OperationThread(thread_id, store1)); } - // Wait for some RMWs to complete in v1 + // Wait for some operations to complete in v1 await Task.Delay(500); - // Initiate checkpoint concurrent to the RMW operation threads + // Initiate checkpoint concurrent to the operation threads var task = store1.TakeFullCheckpointAsync(checkpointType); // Wait for the checkpoint to complete @@ -114,12 +83,12 @@ public async ValueTask CheckpointVersionSwitchTest1( (var status, token) = task.AsTask().GetAwaiter().GetResult(); } - // Wait for some RMWs to complete in v2 + // Wait for some operations to complete in v2 await Task.Delay(500); - // Signal RMW threads to stop - rmwDone = true; - await Task.WhenAll(rmwTasks); + // Signal operation threads to stop, and wait for them to finish + opsDone = true; + await Task.WhenAll(opTasks); // Verify the final state of the old store using var s1 = store1.NewSession(new SumFunctions()); @@ -190,5 +159,138 @@ public async ValueTask CheckpointVersionSwitchTest1( ClassicAssert.AreEqual(expectedV1Count[key], output, $"output = {output}"); } } + + public class SumFunctions : SimpleSimpleFunctions + { + public SumFunctions() : base((l, r) => l + r) { } + } + } + + [TestFixture] + public class CheckpointVersionSwitchRmw : StateMachineDriverTestsBase + { + [SetUp] + public void Setup() => BaseSetup(); + + [TearDown] + public void TearDown() => BaseTearDown(); + + protected override void OperationThread(int thread_id, TsavoriteKV store) + { + using var s = store.NewSession(new SumFunctions()); + var bc = s.BasicContext; + var r = new Random(thread_id); + + long key = 0; + long input = 1; + var v1count = new long[numKeys]; + var v2count = new long[numKeys]; + while (!opsDone) + { + // Generate input for RMW + key = r.Next(numKeys); + + // Run the RMW operation + _ = bc.RMW(ref key, ref input); + + // Update expected counts for the old and new version of store + if (bc.Session.Version == 1) + v1count[key]++; + v2count[key]++; + } + + // Update the global expected counts + for (int i = 0; i < numKeys; i++) + { + _ = Interlocked.Add(ref expectedV1Count[i], v1count[i]); + _ = Interlocked.Add(ref expectedV2Count[i], v2count[i]); + } + } + + [Test] + public async ValueTask CheckpointVersionSwitchRmwTest( + [Values(CheckpointType.Snapshot, CheckpointType.FoldOver)] CheckpointType checkpointType, + [Values] bool isAsync, + [Values(1L << 13, 1L << 16)] long indexSize) + => await DoCheckpointVersionSwitchEquivalenceCheck(checkpointType, isAsync, indexSize); + } + + [TestFixture] + public class CheckpointVersionSwitchTxn : StateMachineDriverTestsBase + { + [SetUp] + public void Setup() => BaseSetup(); + + [TearDown] + public void TearDown() => BaseTearDown(); + + protected override void OperationThread(int thread_id, TsavoriteKV store) + { + using var s = store.NewSession(new SumFunctions()); + var lc = s.LockableContext; + var r = new Random(thread_id); + + ClassicAssert.IsTrue(numKeys > 1); + long key1 = 0, key2 = 0; + long input = 1; + var v1count = new long[numKeys]; + var v2count = new long[numKeys]; + while (!opsDone) + { + // Generate input for transaction + key1 = r.Next(numKeys); + do + { + key2 = r.Next(numKeys); + } while (key2 == key1); + + var exclusiveVec = new FixedLengthLockableKeyStruct[] { + new(key1, LockType.Exclusive, lc), + new(key2, LockType.Exclusive, lc) + }; + + // Start transaction, session does not acquire version in this call + lc.BeginLockable(); + + // Lock keys, session acquires version in this call + lc.Lock(exclusiveVec); + + // We have determined the version of the transaction + var txnVersion = lc.Session.Version; + + // Run transaction + _ = lc.RMW(ref key1, ref input); + _ = lc.RMW(ref key2, ref input); + + // Unlock keys + lc.Unlock(exclusiveVec); + + // End transaction + lc.EndLockable(); + + // Update expected counts for the old and new version of store + if (txnVersion == 1) + { + v1count[key1]++; + v1count[key2]++; + } + v2count[key1]++; + v2count[key2]++; + } + + // Update the global expected counts + for (int i = 0; i < numKeys; i++) + { + _ = Interlocked.Add(ref expectedV1Count[i], v1count[i]); + _ = Interlocked.Add(ref expectedV2Count[i], v2count[i]); + } + } + + [Test] + public async ValueTask CheckpointVersionSwitchTxnTest( + [Values(CheckpointType.Snapshot, CheckpointType.FoldOver)] CheckpointType checkpointType, + [Values] bool isAsync, + [Values(1L << 13, 1L << 16)] long indexSize) + => await DoCheckpointVersionSwitchEquivalenceCheck(checkpointType, isAsync, indexSize); } } \ No newline at end of file From 8f72d005096c7dfd3341e82ebc6c2ad6e277a7ab Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 13:32:15 -0800 Subject: [PATCH 26/49] clean the test --- libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs index 2bebc718be..c26489cc51 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineDriverTests.cs @@ -23,7 +23,7 @@ public abstract class StateMachineDriverTestsBase protected bool opsDone; protected long[] expectedV1Count; protected long[] expectedV2Count; - protected readonly int numKeys = 2; + protected readonly int numKeys = 4; protected void BaseSetup() { From d304ecd3a94590ac9adc721867892d6b787f1394 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 15:08:43 -0800 Subject: [PATCH 27/49] cleanup --- .../Index/Checkpointing/FoldOverSMTask.cs | 45 ++++++++++--------- .../Checkpointing/FullCheckpointSMTask.cs | 3 ++ .../HybridLogCheckpointSMTask.cs | 4 ++ .../IncrementalSnapshotCheckpointSMTask.cs | 3 ++ .../Index/Checkpointing/IndexResizeSMTask.cs | 3 ++ .../Checkpointing/SnapshotCheckpointSMTask.cs | 3 ++ .../StreamingSnapshotCheckpointSMTask.cs | 3 ++ 7 files changed, 43 insertions(+), 21 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index e5705cf6c3..6beb417551 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -22,29 +22,32 @@ public FoldOverSMTask(TsavoriteKV sto /// public override void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) { - base.GlobalBeforeEnteringState(next, stateMachineDriver); - - if (next.Phase == Phase.PREPARE) + switch (next.Phase) { - store._lastSnapshotCheckpoint.Dispose(); - } - - if (next.Phase == Phase.IN_PROGRESS) - base.GlobalBeforeEnteringState(next, stateMachineDriver); - - if (next.Phase != Phase.WAIT_FLUSH) return; + case Phase.PREPARE: + store._lastSnapshotCheckpoint.Dispose(); + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; + + case Phase.WAIT_FLUSH: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + try + { + store.epoch.Resume(); + _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); + if (store._hybridLogCheckpoint.flushedSemaphore != null) + stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); + store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; + } + finally + { + store.epoch.Suspend(); + } + break; - try - { - store.epoch.Resume(); - _ = store.hlogBase.ShiftReadOnlyToTail(out var tailAddress, out store._hybridLogCheckpoint.flushedSemaphore); - if (store._hybridLogCheckpoint.flushedSemaphore != null) - stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); - store._hybridLogCheckpoint.info.finalLogicalAddress = tailAddress; - } - finally - { - store.epoch.Suspend(); + default: + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; } } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs index 29d9c1e3d7..8f6f849ba8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs @@ -15,6 +15,7 @@ internal sealed class FullCheckpointSMTask store; readonly Guid guid; + public FullCheckpointSMTask(TsavoriteKV store, Guid guid) { this.store = store; @@ -31,10 +32,12 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); break; + case Phase.WAIT_FLUSH: store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); store._indexCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); break; + case Phase.PERSISTENCE_CALLBACK: store.WriteIndexMetaInfo(); store._indexCheckpoint.Reset(); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index ce4e5d264f..6d3cae09c0 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -42,6 +42,7 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv // Capture begin address before checkpoint starts store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; break; + case Phase.IN_PROGRESS: // Wait for PREPARE threads to finish active transactions and enter barrier while (store.hlogBase.NumActiveLockingSessions > 0) @@ -50,15 +51,18 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv } store.CheckpointVersionShift(lastVersion, next.Version); break; + case Phase.WAIT_FLUSH: store._hybridLogCheckpoint.info.headAddress = store.hlogBase.HeadAddress; store._hybridLogCheckpoint.info.nextVersion = next.Version; break; + case Phase.PERSISTENCE_CALLBACK: CollectMetadata(next, store); store.WriteHybridLogMetaInfo(); store.lastVersion = lastVersion; break; + case Phase.REST: store._hybridLogCheckpoint.Dispose(); var nextTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs index 73ce07bbd6..2b1353c402 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IncrementalSnapshotCheckpointSMTask.cs @@ -29,9 +29,11 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri base.GlobalBeforeEnteringState(next, stateMachineDriver); store._hybridLogCheckpoint.prevVersion = next.Version; break; + case Phase.IN_PROGRESS: base.GlobalBeforeEnteringState(next, stateMachineDriver); break; + case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); @@ -59,6 +61,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri if (store._hybridLogCheckpoint.flushedSemaphore != null) stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); break; + case Phase.PERSISTENCE_CALLBACK: CollectMetadata(next, store); store._hybridLogCheckpoint.info.deltaTailAddress = store._hybridLogCheckpoint.deltaLog.TailAddress; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs index 6fd5c90d78..cb09009440 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSMTask.cs @@ -26,6 +26,7 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state { case Phase.PREPARE_GROW: break; + case Phase.IN_PROGRESS_GROW: // Wait for PREPARE_GROW threads to finish active transactions and enter barrier while (store.hlogBase.NumActiveLockingSessions > 0) @@ -49,9 +50,11 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state store.resizeInfo.version = 1 - store.resizeInfo.version; break; + case Phase.REST: // nothing to do break; + default: throw new TsavoriteException("Invalid Enum Argument"); } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs index 3b5c226eff..86ca051f71 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -29,6 +29,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri base.GlobalBeforeEnteringState(next, stateMachineDriver); store._hybridLogCheckpoint.info.useSnapshotFile = 1; break; + case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); store._hybridLogCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); @@ -68,6 +69,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri if (store._hybridLogCheckpoint.flushedSemaphore != null) stateMachineDriver.AddToWaitingList(store._hybridLogCheckpoint.flushedSemaphore); break; + case Phase.PERSISTENCE_CALLBACK: // Set actual FlushedUntil to the latest possible data in main log that is on disk // If we are using a NullDevice then storage tier is not enabled and FlushedUntilAddress may be ReadOnlyAddress; get all records in memory. @@ -75,6 +77,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri base.GlobalBeforeEnteringState(next, stateMachineDriver); store._lastSnapshotCheckpoint = store._hybridLogCheckpoint.Transfer(); break; + default: base.GlobalBeforeEnteringState(next, stateMachineDriver); break; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index c80a881e69..55433c7c00 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -42,15 +42,18 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._lastSnapshotCheckpoint.Dispose(); store.StreamingSnapshotScanPhase1(); break; + case Phase.PREPARE: store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); base.GlobalBeforeEnteringState(next, stateMachineDriver); break; + case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); var finalLogicalAddress = store.hlogBase.GetTailAddress(); store.StreamingSnapshotScanPhase2(finalLogicalAddress); break; + default: base.GlobalBeforeEnteringState(next, stateMachineDriver); break; From 36a5e18967088c833677f0f33e7d52d96ee835ac Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 16:20:21 -0800 Subject: [PATCH 28/49] Refactor the phases of various machines --- .../core/Index/Checkpointing/Checkpoint.cs | 5 +- .../Index/Checkpointing/FoldOverSMTask.cs | 2 + .../Index/Checkpointing/FullCheckpointSM.cs | 6 --- .../Checkpointing/FullCheckpointSMTask.cs | 53 ------------------- .../HybridLogCheckpointSMTask.cs | 12 ++--- .../Index/Checkpointing/IndexCheckpointSM.cs | 14 +++-- .../Checkpointing/IndexCheckpointSMTask.cs | 25 +++++---- .../Checkpointing/SnapshotCheckpointSMTask.cs | 4 +- .../Index/Checkpointing/StateTransitions.cs | 13 +---- .../StreamingSnapshotCheckpointSM.cs | 3 -- .../StreamingSnapshotCheckpointSMTask.cs | 16 ++---- .../cs/src/core/Index/Recovery/Checkpoint.cs | 6 --- .../Tsavorite/Implementation/InternalRMW.cs | 5 +- .../Implementation/InternalUpsert.cs | 5 +- 14 files changed, 39 insertions(+), 130 deletions(-) delete mode 100644 libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index c53b083420..0c3e54a882 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -13,17 +13,16 @@ public static IStateMachine Full(Tsav { guid = Guid.NewGuid(); var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); - var fullCheckpointTask = new FullCheckpointSMTask(store, guid); if (checkpointType == CheckpointType.FoldOver) { var backend = new FoldOverSMTask(store, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); + return new FullCheckpointSM(targetVersion, indexCheckpointTask, backend); } else if (checkpointType == CheckpointType.Snapshot) { var backend = new SnapshotCheckpointSMTask(store, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask, fullCheckpointTask, backend); + return new FullCheckpointSM(targetVersion, indexCheckpointTask, backend); } else { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index 6beb417551..d0f2610b5c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -26,6 +26,8 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri { case Phase.PREPARE: store._lastSnapshotCheckpoint.Dispose(); + store._hybridLogCheckpointToken = guid; + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); base.GlobalBeforeEnteringState(next, stateMachineDriver); break; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs index 4fe1608fef..f35f63373c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs @@ -24,12 +24,6 @@ public override SystemState NextState(SystemState start) var result = SystemState.Copy(ref start); switch (start.Phase) { - case Phase.REST: - result.Phase = Phase.PREP_INDEX_CHECKPOINT; - break; - case Phase.PREP_INDEX_CHECKPOINT: - result.Phase = Phase.PREPARE; - break; case Phase.IN_PROGRESS: result.Phase = Phase.WAIT_INDEX_CHECKPOINT; break; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs deleted file mode 100644 index 8f6f849ba8..0000000000 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSMTask.cs +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (c) Microsoft Corporation. -// Licensed under the MIT license. - -using System; -using System.Diagnostics; - -namespace Tsavorite.core -{ - /// - /// This task contains logic to orchestrate the index and hybrid log checkpoint in parallel - /// - internal sealed class FullCheckpointSMTask : IStateMachineTask - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - readonly TsavoriteKV store; - readonly Guid guid; - - public FullCheckpointSMTask(TsavoriteKV store, Guid guid) - { - this.store = store; - this.guid = guid; - } - - /// - public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver stateMachineDriver) - { - switch (next.Phase) - { - case Phase.PREP_INDEX_CHECKPOINT: - Debug.Assert(store._hybridLogCheckpoint.IsDefault()); - store._hybridLogCheckpointToken = guid; - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - break; - - case Phase.WAIT_FLUSH: - store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); - store._indexCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); - break; - - case Phase.PERSISTENCE_CALLBACK: - store.WriteIndexMetaInfo(); - store._indexCheckpoint.Reset(); - break; - } - } - - /// - public void GlobalAfterEnteringState(SystemState next, StateMachineDriver stateMachineDriver) - { - } - } -} \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs index 6d3cae09c0..d6a9ef385c 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSMTask.cs @@ -17,7 +17,7 @@ internal abstract class HybridLogCheckpointSMTask store; long lastVersion; - readonly Guid guid; + protected readonly Guid guid; public HybridLogCheckpointSMTask(TsavoriteKV store, Guid guid) { @@ -31,15 +31,9 @@ public virtual void GlobalBeforeEnteringState(SystemState next, StateMachineDriv switch (next.Phase) { case Phase.PREPARE: - lastVersion = next.Version; - if (store._hybridLogCheckpoint.IsDefault()) - { - store._hybridLogCheckpointToken = guid; - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - } - store._hybridLogCheckpoint.info.version = next.Version; + // Capture state before checkpoint starts + lastVersion = store._hybridLogCheckpoint.info.version = next.Version; store._hybridLogCheckpoint.info.startLogicalAddress = store.hlogBase.GetTailAddress(); - // Capture begin address before checkpoint starts store._hybridLogCheckpoint.info.beginAddress = store.hlogBase.BeginAddress; break; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs index b1223afd97..1a300e9bbf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs @@ -23,12 +23,18 @@ public override SystemState NextState(SystemState start) switch (start.Phase) { case Phase.REST: - result.Phase = Phase.PREP_INDEX_CHECKPOINT; + result.Phase = Phase.PREPARE; break; - case Phase.PREP_INDEX_CHECKPOINT: - result.Phase = Phase.WAIT_INDEX_ONLY_CHECKPOINT; + case Phase.PREPARE: + result.Phase = Phase.WAIT_INDEX_CHECKPOINT; break; - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: + case Phase.WAIT_INDEX_CHECKPOINT: + result.Phase = Phase.WAIT_FLUSH; + break; + case Phase.WAIT_FLUSH: + result.Phase = Phase.PERSISTENCE_CALLBACK; + break; + case Phase.PERSISTENCE_CALLBACK: result.Phase = Phase.REST; break; default: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs index be80508318..c3de24f08e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSMTask.cs @@ -27,7 +27,7 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state { switch (next.Phase) { - case Phase.PREP_INDEX_CHECKPOINT: + case Phase.PREPARE: Debug.Assert(store._indexCheckpoint.IsDefault()); store._indexCheckpointToken = guid; store.InitializeIndexCheckpoint(store._indexCheckpointToken); @@ -36,21 +36,20 @@ public void GlobalBeforeEnteringState(SystemState next, StateMachineDriver state break; case Phase.WAIT_INDEX_CHECKPOINT: - case Phase.WAIT_INDEX_ONLY_CHECKPOINT: store.AddIndexCheckpointWaitingList(stateMachineDriver); break; - case Phase.REST: - // If the tail address has already been obtained, because another task on the state machine - // has done so earlier (e.g. FullCheckpoint captures log tail at WAIT_FLUSH), don't update - // the tail address. - if (store.ObtainCurrentTailAddress(ref store._indexCheckpoint.info.finalLogicalAddress)) - store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); - if (!store._indexCheckpoint.IsDefault()) - { - store.WriteIndexMetaInfo(); - store._indexCheckpoint.Reset(); - } + case Phase.WAIT_FLUSH: + store._indexCheckpoint.info.num_buckets = store.overflowBucketsAllocator.GetMaxValidAddress(); + store._indexCheckpoint.info.finalLogicalAddress = store.hlogBase.GetTailAddress(); + break; + + case Phase.PERSISTENCE_CALLBACK: + store.WriteIndexMetaInfo(); + store._indexCheckpoint.Reset(); + break; + + default: break; } } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs index 86ca051f71..687b940b0d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/SnapshotCheckpointSMTask.cs @@ -26,8 +26,10 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri { case Phase.PREPARE: store._lastSnapshotCheckpoint.Dispose(); - base.GlobalBeforeEnteringState(next, stateMachineDriver); + store._hybridLogCheckpointToken = guid; + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); store._hybridLogCheckpoint.info.useSnapshotFile = 1; + base.GlobalBeforeEnteringState(next, stateMachineDriver); break; case Phase.WAIT_FLUSH: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs index 0579fc5d63..a28c10dbfd 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateTransitions.cs @@ -29,10 +29,10 @@ public enum Phase : int /// In-progress phase, entering (v+1) version IN_PROGRESS, - /// Wait for an index checkpoint to finish + /// Wait for an index checkpoint flush to finish WAIT_INDEX_CHECKPOINT, - /// Wait for data flush to complete + /// Wait for hybrid log flush to complete WAIT_FLUSH, /// After flush has completed, write metadata to persistent storage and issue user callbacks @@ -41,15 +41,6 @@ public enum Phase : int /// The default phase; no state-machine operation is operating REST, - /// Prepare for an index checkpoint - PREP_INDEX_CHECKPOINT, - - /// Wait for an index-only checkpoint to complete - WAIT_INDEX_ONLY_CHECKPOINT, - - /// Wait for pre-scan (until ReadOnlyAddress) to complete for streaming snapshot - PREP_STREAMING_SNAPSHOT_CHECKPOINT, - /// Prepare for a checkpoint, still in (v) version PREPARE, diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs index 9656df1a11..f8e74f3176 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs @@ -23,9 +23,6 @@ public override SystemState NextState(SystemState start) switch (start.Phase) { case Phase.REST: - result.Phase = Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT; - break; - case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: result.Phase = Phase.PREPARE; break; case Phase.IN_PROGRESS: diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index 55433c7c00..b4d99098c6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -2,8 +2,6 @@ // Licensed under the MIT license. using System; -using System.Threading; -using System.Threading.Tasks; namespace Tsavorite.core { @@ -20,13 +18,11 @@ sealed class StreamingSnapshotCheckpointSMTask { readonly long targetVersion; - readonly Guid guid; public StreamingSnapshotCheckpointSMTask(long targetVersion, TsavoriteKV store, Guid guid) : base(store, guid) { this.targetVersion = targetVersion; - this.guid = guid; } /// @@ -34,22 +30,16 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri { switch (next.Phase) { - case Phase.PREP_STREAMING_SNAPSHOT_CHECKPOINT: - base.GlobalBeforeEnteringState(next, stateMachineDriver); + case Phase.PREPARE: + store._lastSnapshotCheckpoint.Dispose(); store._hybridLogCheckpointToken = guid; + store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); store._hybridLogCheckpoint.info.version = next.Version; store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; - store._lastSnapshotCheckpoint.Dispose(); store.StreamingSnapshotScanPhase1(); break; - case Phase.PREPARE: - store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - base.GlobalBeforeEnteringState(next, stateMachineDriver); - break; - case Phase.WAIT_FLUSH: - base.GlobalBeforeEnteringState(next, stateMachineDriver); var finalLogicalAddress = store.hlogBase.GetTailAddress(); store.StreamingSnapshotScanPhase2(finalLogicalAddress); break; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs index 27daee2063..c70ba6f2ff 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Recovery/Checkpoint.cs @@ -75,12 +75,6 @@ internal void WriteIndexMetaInfo() checkpointManager.CommitIndexCheckpoint(_indexCheckpointToken, _indexCheckpoint.info.ToByteArray()); } - internal bool ObtainCurrentTailAddress(ref long location) - { - var tailAddress = hlogBase.GetTailAddress(); - return Interlocked.CompareExchange(ref location, tailAddress, 0) == 0; - } - internal void InitializeIndexCheckpoint(Guid indexToken) { _indexCheckpoint.Initialize(indexToken, state[resizeInfo.version].size, checkpointManager); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs index abd72e03fe..c7cab53892 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalRMW.cs @@ -300,9 +300,6 @@ private LatchDestination CheckCPRConsistencyRMW(Phase phase, ref OperationStackC switch (phase) { - case Phase.PREPARE: // Thread is in V - break; - case Phase.IN_PROGRESS: // Thread is in v+1 case Phase.WAIT_INDEX_CHECKPOINT: case Phase.WAIT_FLUSH: @@ -313,7 +310,7 @@ private LatchDestination CheckCPRConsistencyRMW(Phase phase, ref OperationStackC return LatchDestination.CreateNewRecord; // Record is in memory so force creation of a (V+1) record break; // Normal Processing; the record is below HeadAddress so the operation will go pending - default: + default: // Thread is in V break; } return LatchDestination.NormalProcessing; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs index bd045e8037..e894654ae7 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Implementation/InternalUpsert.cs @@ -265,9 +265,6 @@ private LatchDestination CheckCPRConsistencyUpsert(Phase phase, ref OperationSta switch (phase) { - case Phase.PREPARE: // Thread is in V - break; - case Phase.IN_PROGRESS: // Thread is in V+1 case Phase.WAIT_INDEX_CHECKPOINT: case Phase.WAIT_FLUSH: @@ -275,7 +272,7 @@ private LatchDestination CheckCPRConsistencyUpsert(Phase phase, ref OperationSta break; // Normal Processing; V+1 thread encountered a record in V+1 return LatchDestination.CreateNewRecord; // Upsert never goes pending; always force creation of a (V+1) record - default: + default: // Thread is in V break; } return LatchDestination.NormalProcessing; From 0c689888c6f622afdbff1ace880554ad9d3a17d1 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 17:32:24 -0800 Subject: [PATCH 29/49] format --- .../Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs index d0f2610b5c..4b599c9dbf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FoldOverSMTask.cs @@ -30,7 +30,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); base.GlobalBeforeEnteringState(next, stateMachineDriver); break; - + case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); try From efd8bd2f24f24f1d1584ff437ca72a50a2a95c00 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Mon, 3 Mar 2025 18:48:49 -0800 Subject: [PATCH 30/49] initial commit --- libs/server/Transaction/TransactionManager.cs | 38 +++++-- .../src/core/ClientSession/ClientSession.cs | 21 ++++ .../core/ClientSession/ILockableContext.cs | 5 + .../src/core/ClientSession/LockableContext.cs | 3 + .../ClientSession/LockableUnsafeContext.cs | 3 + .../core/Index/Checkpointing/Checkpoint.cs | 104 ++++++++++++++++++ .../StreamingSnapshotCheckpointSM.cs | 4 +- 7 files changed, 166 insertions(+), 12 deletions(-) diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 1119cf84db..0c131b06b1 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -292,6 +292,33 @@ internal void GetKeysForValidation(byte* recvBufferPtr, out ArgSlice[] keys, out readOnly = keyEntries.IsReadOnly; } + void BeginLockable(StoreType transactionStoreType) + { + while (true) + { + if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Main) + { + lockableContext.BeginLockable(); + } + if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Object) + { + if (objectStoreBasicContext.IsNull) + throw new Exception("Trying to perform object store transaction with object store disabled"); + if (objectStoreLockableContext.TryBeginLockable()) + { + // If we managed to begin lockable for the object store, we MUST be in the same version as the main store + return; + } + objectStoreLockableContext.Refresh(); + } + if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Main) + { + lockableContext.EndLockable(); + lockableContext.Refresh(); + } + } + } + internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, TimeSpan lock_timeout = default) { // Save watch keys to lock list @@ -299,16 +326,7 @@ internal bool Run(bool internal_txn = false, bool fail_fast_on_lock = false, Tim watchContainer.SaveKeysToLock(this); // Acquire lock sessions - if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Main) - { - lockableContext.BeginLockable(); - } - if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Object) - { - if (objectStoreBasicContext.IsNull) - throw new Exception("Trying to perform object store transaction with object store disabled"); - objectStoreLockableContext.BeginLockable(); - } + BeginLockable(transactionStoreType); bool lockSuccess; if (fail_fast_on_lock) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 9d1c5d5d92..199ad61056 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -69,6 +69,27 @@ internal void AcquireLockable(TSessionFunctions sessionFuncti } } + internal bool TryAcquireLockable(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper + { + CheckIsNotAcquiredLockable(); + + // Checkpoints cannot complete while we have active locking sessions. + if (IsInPreparePhase()) + { + return false; + } + + store.IncrementNumLockingSessions(); + isAcquiredLockable = true; + + if (!IsInPreparePhase()) + return true; + + InternalReleaseLockable(); + return false; + } + internal void ReleaseLockable() { CheckIsAcquiredLockable(); diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs index 373c56ae40..e2c455675c 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ILockableContext.cs @@ -19,6 +19,11 @@ public interface ILockableContext /// void BeginLockable(); + /// + /// Try to begin a series of lock operations on possibly multiple keys; call before any locks are taken. + /// + bool TryBeginLockable(); + /// /// Ends a series of lock operations on possibly multiple keys; call after all locks are released. /// diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs index c5cab39895..361a66f11c 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs @@ -36,6 +36,9 @@ internal LockableContext(ClientSession public void BeginLockable() => clientSession.AcquireLockable(sessionFunctions); + /// + public bool TryBeginLockable() => clientSession.TryAcquireLockable(sessionFunctions); + /// public void EndLockable() => clientSession.ReleaseLockable(); diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs index 3eb6e02188..adc0074b23 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs @@ -47,6 +47,9 @@ internal LockableUnsafeContext(ClientSession public void BeginLockable() => clientSession.AcquireLockable(sessionFunctions); + /// + public bool TryBeginLockable() => clientSession.TryAcquireLockable(sessionFunctions); + /// public void EndLockable() => clientSession.ReleaseLockable(); #endregion Begin/EndLockable diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 0c3e54a882..5869591cec 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -30,6 +30,52 @@ public static IStateMachine Full(Tsav } } + public static IStateMachine Full( + TsavoriteKV store1, + TsavoriteKV store2, + CheckpointType checkpointType, long targetVersion, out Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator + { + guid = Guid.NewGuid(); + var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); + var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + + if (checkpointType == CheckpointType.FoldOver) + { + var backend1 = new FoldOverSMTask(store1, guid); + var backend2 = new FoldOverSMTask(store2, guid); + return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + } + else if (checkpointType == CheckpointType.Snapshot) + { + var backend1 = new SnapshotCheckpointSMTask(store1, guid); + var backend2 = new SnapshotCheckpointSMTask(store2, guid); + return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + } + else + { + throw new TsavoriteException("Invalid checkpoint type"); + } + } + + public static IStateMachine Streaming( + TsavoriteKV store1, + TsavoriteKV store2, + long targetVersion, out Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator + { + guid = Guid.NewGuid(); + var backend1 = new StreamingSnapshotCheckpointSMTask(targetVersion, store1, guid); + var backend2 = new StreamingSnapshotCheckpointSMTask(targetVersion, store2, guid); + return new StreamingSnapshotCheckpointSM(targetVersion, backend1, backend2); + } + public static IStateMachine Streaming(TsavoriteKV store, long targetVersion, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator @@ -48,6 +94,21 @@ public static IStateMachine IndexOnly return new IndexCheckpointSM(targetVersion, indexCheckpointTask); } + public static IStateMachine IndexOnly( + TsavoriteKV store1, + TsavoriteKV store2, + long targetVersion, out Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator + { + guid = Guid.NewGuid(); + var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); + var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + return new IndexCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2); + } + public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator @@ -70,6 +131,35 @@ public static IStateMachine HybridLogOnly( + TsavoriteKV store1, + TsavoriteKV store2, + CheckpointType checkpointType, long targetVersion, out Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator + { + guid = Guid.NewGuid(); + + if (checkpointType == CheckpointType.FoldOver) + { + var backend1 = new FoldOverSMTask(store1, guid); + var backend2 = new FoldOverSMTask(store2, guid); + return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + } + else if (checkpointType == CheckpointType.Snapshot) + { + var backend1 = new SnapshotCheckpointSMTask(store1, guid); + var backend2 = new SnapshotCheckpointSMTask(store2, guid); + return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + } + else + { + throw new TsavoriteException("Invalid checkpoint type"); + } + } + public static IStateMachine IncrementalHybridLogOnly(TsavoriteKV store, long targetVersion, Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator @@ -77,5 +167,19 @@ public static IStateMachine IncrementalHybridLogOnly(store, guid); return new HybridLogCheckpointSM(targetVersion, backend); } + + public static IStateMachine IncrementalHybridLogOnly( + TsavoriteKV store1, + TsavoriteKV store2, + CheckpointType checkpointType, long targetVersion, Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator + { + var backend1 = new IncrementalSnapshotCheckpointSMTask(store1, guid); + var backend2 = new IncrementalSnapshotCheckpointSMTask(store2, guid); + return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + } } } \ No newline at end of file diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs index f8e74f3176..cf18ba1346 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs @@ -12,8 +12,8 @@ class StreamingSnapshotCheckpointSM : VersionChangeSM /// Construct a new StreamingSnapshotCheckpointStateMachine, drawing boundary at targetVersion. /// /// upper limit (inclusive) of the version included - public StreamingSnapshotCheckpointSM(long targetVersion, IStateMachineTask backend) - : base(targetVersion, backend) + public StreamingSnapshotCheckpointSM(long targetVersion, params IStateMachineTask[] tasks) + : base(targetVersion, tasks) { } /// From 511fe75ed2fe44ed2b8ad5f3a459b7bdb61099ee Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 5 Mar 2025 16:00:25 -0800 Subject: [PATCH 31/49] remove sessionName --- .../cs/src/core/ClientSession/IClientSession.cs | 1 - .../cs/src/core/ClientSession/ManageClientSessions.cs | 10 +++------- .../cs/src/core/Index/Common/ExecutionContext.cs | 4 +--- 3 files changed, 4 insertions(+), 11 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs index 2386b5e116..d252684165 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/IClientSession.cs @@ -5,7 +5,6 @@ namespace Tsavorite.core { class SessionInfo { - public string sessionName; public bool isActive; public IClientSession session; } diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs index 4b344be654..e98466b133 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ManageClientSessions.cs @@ -17,20 +17,16 @@ public unsafe partial class TsavoriteKV /// Callback functions - /// Name of session (optional) /// for this session; override those specified at TsavoriteKV level, and may be overridden on individual Read operations /// Session instance - public ClientSession NewSession(TFunctions functions, string sessionName = null, - ReadCopyOptions readCopyOptions = default) + public ClientSession NewSession(TFunctions functions, ReadCopyOptions readCopyOptions = default) where TFunctions : ISessionFunctions { if (functions == null) throw new ArgumentNullException(nameof(functions)); - if (sessionName == "") - throw new TsavoriteException("Cannot use empty string as session name"); int sessionID = Interlocked.Increment(ref maxSessionID); - var ctx = new TsavoriteExecutionContext(sessionID, sessionName); + var ctx = new TsavoriteExecutionContext(sessionID); ctx.MergeReadCopyOptions(ReadCopyOptions, readCopyOptions); @@ -41,7 +37,7 @@ public ClientSession : Ts internal sealed class TsavoriteExecutionContext { internal readonly int sessionID; - internal readonly string sessionName; // Control automatic Read copy operations. These flags override flags specified at the TsavoriteKV level, but may be overridden on the individual Read() operations internal ReadCopyOptions ReadCopyOptions; @@ -31,11 +30,10 @@ internal sealed class TsavoriteExecutionContext public int asyncPendingCount; internal RevivificationStats RevivificationStats = new(); - public TsavoriteExecutionContext(int sessionID, string sessionName) + public TsavoriteExecutionContext(int sessionID) { SessionState = SystemState.Make(Phase.REST, 1); this.sessionID = sessionID; - this.sessionName = sessionName; readyResponses = new AsyncQueue>(); ioPendingRequests = new Dictionary>(); pendingReads = new AsyncCountDown(); From 8ed6101e549c5f9005e885f306bdffd266bc511d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 5 Mar 2025 16:02:12 -0800 Subject: [PATCH 32/49] update LightEpoch based on PR comment --- libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs index 8a5e3022b7..8caa6a1ec6 100644 --- a/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs +++ b/libs/storage/Tsavorite/cs/src/core/Epochs/LightEpoch.cs @@ -477,10 +477,14 @@ struct Entry public int reentrant; [FieldOffset(16)] - public fixed long placeholder[6]; + public fixed long padding[6]; // Padding to end of cache line public override string ToString() => $"lce = {localCurrentEpoch}, tid = {threadId}, re-ent {reentrant}"; } + + /// + /// Pair of epoch and action to be executed + /// struct EpochActionPair { public long epoch; From 17f04ac380cb67266bbb29e7a4951a0e2fd8a76d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 5 Mar 2025 16:48:25 -0800 Subject: [PATCH 33/49] fix break --- libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs | 2 +- libs/storage/Tsavorite/cs/test/ThreadSession.cs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index 9bb60e1d12..a798d87d9a 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -119,7 +119,7 @@ void Prepare(out SimpleFunctions f, NumClicks value; - s1 = store.NewSession(f, "foo"); + s1 = store.NewSession(f); var bc1 = s1.BasicContext; for (int key = 0; key < NumOps; key++) diff --git a/libs/storage/Tsavorite/cs/test/ThreadSession.cs b/libs/storage/Tsavorite/cs/test/ThreadSession.cs index 7d50f6c7ab..f9a6e17b72 100644 --- a/libs/storage/Tsavorite/cs/test/ThreadSession.cs +++ b/libs/storage/Tsavorite/cs/test/ThreadSession.cs @@ -66,7 +66,7 @@ public void Dispose() private void SecondSession() { - s2 = store.NewSession(f, null); + s2 = store.NewSession(f); uc2 = s2.UnsafeContext; uc2.BeginUnsafe(); @@ -144,7 +144,7 @@ public void getLUC() private void LUCThread() { - session = store.NewSession(f, null); + session = store.NewSession(f); _ = ev.Set(); while (true) From 6dfc383719d2be0e8fc6ef5898a7875f018eafb4 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 5 Mar 2025 16:52:24 -0800 Subject: [PATCH 34/49] Use session-local isAcquiredLockable as signal for threads to decide whether to spin or operate in PREPARE phase. Moved isAcquiredLockable to Ctx. This commit also removes a race that will re-establish the invariant that no threads are operating in PREPARE while any thread is operating in IN_PROGRESS phase. --- .../src/core/ClientSession/ClientSession.cs | 30 ++++++++++--------- .../src/core/ClientSession/LockableContext.cs | 10 +++---- .../ClientSession/LockableUnsafeContext.cs | 10 +++---- .../src/core/Index/Common/ExecutionContext.cs | 2 ++ .../core/Index/Tsavorite/TsavoriteThread.cs | 14 ++++----- 5 files changed, 35 insertions(+), 31 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs index 9d1c5d5d92..56296b2438 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/ClientSession.cs @@ -40,14 +40,12 @@ public sealed class ClientSession scanCursorState; internal void AcquireLockable(TSessionFunctions sessionFunctions) where TSessionFunctions : ISessionFunctionsWrapper { - CheckIsNotAcquiredLockable(); + CheckIsNotAcquiredLockable(sessionFunctions); while (true) { @@ -60,39 +58,43 @@ internal void AcquireLockable(TSessionFunctions sessionFuncti } store.IncrementNumLockingSessions(); - isAcquiredLockable = true; + sessionFunctions.Ctx.isAcquiredLockable = true; if (!IsInPreparePhase()) break; - InternalReleaseLockable(); + InternalReleaseLockable(sessionFunctions); _ = Thread.Yield(); } } - internal void ReleaseLockable() + internal void ReleaseLockable(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - CheckIsAcquiredLockable(); + CheckIsAcquiredLockable(sessionFunctions); if (TotalLockCount > 0) throw new TsavoriteException($"EndLockable called with locks held: {sharedLockCount} shared locks, {exclusiveLockCount} exclusive locks"); - InternalReleaseLockable(); + InternalReleaseLockable(sessionFunctions); } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void InternalReleaseLockable() + private void InternalReleaseLockable(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - isAcquiredLockable = false; + sessionFunctions.Ctx.isAcquiredLockable = false; store.DecrementNumLockingSessions(); } - internal void CheckIsAcquiredLockable() + internal void CheckIsAcquiredLockable(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - if (!isAcquiredLockable) + if (!sessionFunctions.Ctx.isAcquiredLockable) throw new TsavoriteException("Lockable method call when BeginLockable has not been called"); } - void CheckIsNotAcquiredLockable() + void CheckIsNotAcquiredLockable(TSessionFunctions sessionFunctions) + where TSessionFunctions : ISessionFunctionsWrapper { - if (isAcquiredLockable) + if (sessionFunctions.Ctx.isAcquiredLockable) throw new TsavoriteException("BeginLockable cannot be called twice (call EndLockable first)"); } diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs index c5cab39895..5658018f4f 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableContext.cs @@ -37,7 +37,7 @@ internal LockableContext(ClientSession clientSession.AcquireLockable(sessionFunctions); /// - public void EndLockable() => clientSession.ReleaseLockable(); + public void EndLockable() => clientSession.ReleaseLockable(sessionFunctions); #endregion Begin/EndLockable @@ -236,7 +236,7 @@ internal static void DoManualUnlock(ClientSession(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); bool lockAquired = false; while (!lockAquired) @@ -287,7 +287,7 @@ public bool TryLock(TLockableKey[] keys, TimeSpan timeout, Cancell public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); clientSession.UnsafeResumeThread(sessionFunctions); @@ -320,7 +320,7 @@ public bool TryPromoteLock(TLockableKey key, CancellationToken can public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Lock()"); clientSession.UnsafeResumeThread(sessionFunctions); @@ -341,7 +341,7 @@ public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, Can public void Unlock(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(!clientSession.store.epoch.ThisInstanceProtected(), "Trying to protect an already-protected epoch for LockableUnsafeContext.Unlock()"); clientSession.UnsafeResumeThread(sessionFunctions); diff --git a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs index 3eb6e02188..9dc1fb2c23 100644 --- a/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/ClientSession/LockableUnsafeContext.cs @@ -48,7 +48,7 @@ internal LockableUnsafeContext(ClientSession clientSession.AcquireLockable(sessionFunctions); /// - public void EndLockable() => clientSession.ReleaseLockable(); + public void EndLockable() => clientSession.ReleaseLockable(sessionFunctions); #endregion Begin/EndLockable #region Key Locking @@ -72,7 +72,7 @@ internal LockableUnsafeContext(ClientSession(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); while (true) { @@ -120,7 +120,7 @@ public bool TryLock(TLockableKey[] keys, TimeSpan timeout, Cancell public bool TryLock(TLockableKey[] keys, int start, int count, TimeSpan timeout, CancellationToken cancellationToken) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); return LockableContext.DoManualTryLock(sessionFunctions, clientSession, keys, start, count, timeout, cancellationToken); @@ -145,7 +145,7 @@ public bool TryPromoteLock(TLockableKey key, CancellationToken can public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, CancellationToken cancellationToken) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Lock()"); return LockableContext.DoManualTryPromoteLock(sessionFunctions, clientSession, key, timeout, cancellationToken); @@ -158,7 +158,7 @@ public bool TryPromoteLock(TLockableKey key, TimeSpan timeout, Can public void Unlock(TLockableKey[] keys, int start, int count) where TLockableKey : ILockableKey { - clientSession.CheckIsAcquiredLockable(); + clientSession.CheckIsAcquiredLockable(sessionFunctions); Debug.Assert(clientSession.store.epoch.ThisInstanceProtected(), "Epoch protection required for LockableUnsafeContext.Unlock()"); LockableContext.DoManualUnlock(clientSession, keys, start, start + count - 1); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs index 56fa85ea10..84f7957b4b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/ExecutionContext.cs @@ -29,6 +29,7 @@ internal sealed class TsavoriteExecutionContext public readonly AsyncQueue> readyResponses; public int asyncPendingCount; internal RevivificationStats RevivificationStats = new(); + public bool isAcquiredLockable; public TsavoriteExecutionContext(int sessionID) { @@ -37,6 +38,7 @@ public TsavoriteExecutionContext(int sessionID) readyResponses = new AsyncQueue>(); ioPendingRequests = new Dictionary>(); pendingReads = new AsyncCountDown(); + isAcquiredLockable = false; } public int SyncIoPendingCount => ioPendingRequests.Count - asyncPendingCount; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 9f4fed51c5..e6e96ef492 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -19,7 +19,7 @@ internal void InternalRefresh Date: Wed, 5 Mar 2025 17:18:39 -0800 Subject: [PATCH 35/49] address review comments --- .../Index/Checkpointing/StateMachineDriver.cs | 4 ++-- ...ine.cs => TsavoriteStateMachineProperties.cs} | 0 libs/storage/Tsavorite/cs/test/RecoveryChecks.cs | 2 +- .../cs/test/StateMachineBarrierTests.cs | 16 ++++++++-------- 4 files changed, 11 insertions(+), 11 deletions(-) rename libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/{TsavoriteStateMachine.cs => TsavoriteStateMachineProperties.cs} (100%) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index 6653b0f075..ea05c8fc19 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -118,7 +118,7 @@ void GlobalStateMachineStep(SystemState expectedState) systemState.Word = nextState.Word; // Release waiters for new phase - waitForTransitionOut?.Release(int.MaxValue); + _ = waitForTransitionOut?.Release(int.MaxValue); // Write new semaphore waitForTransitionOut = new SemaphoreSlim(0); @@ -136,7 +136,7 @@ void GlobalStateMachineStep(SystemState expectedState) { epoch.Suspend(); } - waitingList.Add(waitForTransitionIn); + AddToWaitingList(waitForTransitionIn); } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs similarity index 100% rename from libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachine.cs rename to libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/TsavoriteStateMachineProperties.cs diff --git a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs index abde942ea1..4df0676d56 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs @@ -734,7 +734,7 @@ public async ValueTask RecoveryCheck5( _ = bc1.CompletePending(true); } - var result = store1.GrowIndexAsync().GetAwaiter().GetResult(); + var result = await store1.GrowIndexAsync(); ClassicAssert.IsTrue(result); for (long key = 0; key < 1000; key++) diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index a798d87d9a..13e695812c 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -98,13 +98,13 @@ public void StateMachineBarrierTest1() s1.Dispose(); } - void Prepare(out SimpleFunctions f, - out ClientSession s1, - out UnsafeContext uc1, - out ThreadSession s2, + void Prepare(out SMSimpleFunctions f, + out ClientSession s1, + out UnsafeContext uc1, + out ThreadSession s2, long toVersion = -1) { - f = new SimpleFunctions(); + f = new SMSimpleFunctions(); // We should be in REST, 1 ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); @@ -119,7 +119,7 @@ void Prepare(out SimpleFunctions f, NumClicks value; - s1 = store.NewSession(f); + s1 = store.NewSession(f); var bc1 = s1.BasicContext; for (int key = 0; key < NumOps; key++) @@ -136,7 +136,7 @@ void Prepare(out SimpleFunctions f, uc1.BeginUnsafe(); // Start session s2 on another thread for testing - s2 = store.CreateThreadSession(f); + s2 = store.CreateThreadSession(f); // We should be in REST, 1 ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); @@ -152,7 +152,7 @@ void Prepare(out SimpleFunctions f, } } - public class SimpleFunctions : SimpleSessionFunctions + public class SMSimpleFunctions : SimpleSessionFunctions { public override void ReadCompletionCallback(ref AdId key, ref NumClicks input, ref NumClicks output, Empty ctx, Status status, RecordMetadata recordMetadata) { From 3e3a377bcb015997b8c69939d4533f3acf4ad601 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Wed, 5 Mar 2025 17:24:09 -0800 Subject: [PATCH 36/49] nit --- .../Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index e6e96ef492..203a8db06f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -35,7 +35,7 @@ internal void InternalRefresh Date: Fri, 7 Mar 2025 10:50:52 -0800 Subject: [PATCH 37/49] minor code move --- .../core/Index/Checkpointing/Checkpoint.cs | 112 +++++++++--------- 1 file changed, 58 insertions(+), 54 deletions(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 5869591cec..5f94cad543 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -7,6 +7,7 @@ namespace Tsavorite.core { static class Checkpoint { + #region Single-store APIs public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator @@ -30,30 +31,45 @@ public static IStateMachine Full(Tsav } } - public static IStateMachine Full( + public static IStateMachine Streaming( TsavoriteKV store1, TsavoriteKV store2, - CheckpointType checkpointType, long targetVersion, out Guid guid) + long targetVersion, out Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); - var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + var backend1 = new StreamingSnapshotCheckpointSMTask(targetVersion, store1, guid); + var backend2 = new StreamingSnapshotCheckpointSMTask(targetVersion, store2, guid); + return new StreamingSnapshotCheckpointSM(targetVersion, backend1, backend2); + } + + public static IStateMachine IndexOnly(TsavoriteKV store, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); + var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); + return new IndexCheckpointSM(targetVersion, indexCheckpointTask); + } + + public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + guid = Guid.NewGuid(); if (checkpointType == CheckpointType.FoldOver) { - var backend1 = new FoldOverSMTask(store1, guid); - var backend2 = new FoldOverSMTask(store2, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + var backend = new FoldOverSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); } else if (checkpointType == CheckpointType.Snapshot) { - var backend1 = new SnapshotCheckpointSMTask(store1, guid); - var backend2 = new SnapshotCheckpointSMTask(store2, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + var backend = new SnapshotCheckpointSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); } else { @@ -61,19 +77,45 @@ public static IStateMachine Full( + public static IStateMachine IncrementalHybridLogOnly(TsavoriteKV store, long targetVersion, Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator + { + var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); + return new HybridLogCheckpointSM(targetVersion, backend); + } + #endregion + + #region Two-store APIs + public static IStateMachine Full( TsavoriteKV store1, TsavoriteKV store2, - long targetVersion, out Guid guid) + CheckpointType checkpointType, long targetVersion, out Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var backend1 = new StreamingSnapshotCheckpointSMTask(targetVersion, store1, guid); - var backend2 = new StreamingSnapshotCheckpointSMTask(targetVersion, store2, guid); - return new StreamingSnapshotCheckpointSM(targetVersion, backend1, backend2); + var indexCheckpointTask1 = new IndexCheckpointSMTask(store1, guid); + var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); + + if (checkpointType == CheckpointType.FoldOver) + { + var backend1 = new FoldOverSMTask(store1, guid); + var backend2 = new FoldOverSMTask(store2, guid); + return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + } + else if (checkpointType == CheckpointType.Snapshot) + { + var backend1 = new SnapshotCheckpointSMTask(store1, guid); + var backend2 = new SnapshotCheckpointSMTask(store2, guid); + return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + } + else + { + throw new TsavoriteException("Invalid checkpoint type"); + } } public static IStateMachine Streaming(TsavoriteKV store, long targetVersion, out Guid guid) @@ -85,15 +127,6 @@ public static IStateMachine Streaming return new StreamingSnapshotCheckpointSM(targetVersion, backend); } - public static IStateMachine IndexOnly(TsavoriteKV store, long targetVersion, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - guid = Guid.NewGuid(); - var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); - return new IndexCheckpointSM(targetVersion, indexCheckpointTask); - } - public static IStateMachine IndexOnly( TsavoriteKV store1, TsavoriteKV store2, @@ -109,28 +142,6 @@ public static IStateMachine IndexOnly(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - guid = Guid.NewGuid(); - - if (checkpointType == CheckpointType.FoldOver) - { - var backend = new FoldOverSMTask(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); - } - else if (checkpointType == CheckpointType.Snapshot) - { - var backend = new SnapshotCheckpointSMTask(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); - } - else - { - throw new TsavoriteException("Invalid checkpoint type"); - } - } - public static IStateMachine HybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, @@ -160,14 +171,6 @@ public static IStateMachine HybridLogOnly(TsavoriteKV store, long targetVersion, Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator - { - var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); - } - public static IStateMachine IncrementalHybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, @@ -181,5 +184,6 @@ public static IStateMachine IncrementalHybridLogOnly(store2, guid); return new HybridLogCheckpointSM(targetVersion, backend1, backend2); } + #endregion } } \ No newline at end of file From 443b8abeacba4fbf09205730de7be18b801b6d2d Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 11:58:30 -0800 Subject: [PATCH 38/49] use shared epoch across stores --- libs/host/GarnetServer.cs | 3 +- libs/server/Servers/GarnetServerOptions.cs | 39 +++++++++++++++---- .../Index/Checkpointing/StateMachineDriver.cs | 4 +- .../cs/src/core/Index/Common/KVSettings.cs | 5 +++ .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 4 +- .../src/core/Index/Tsavorite/TsavoriteBase.cs | 4 +- 6 files changed, 44 insertions(+), 15 deletions(-) diff --git a/libs/host/GarnetServer.cs b/libs/host/GarnetServer.cs index dd5da8541f..914f4318df 100644 --- a/libs/host/GarnetServer.cs +++ b/libs/host/GarnetServer.cs @@ -227,6 +227,7 @@ private void InitializeServer() if (!setMax && !ThreadPool.SetMaxThreads(maxThreads, maxCPThreads)) throw new Exception($"Unable to call ThreadPool.SetMaxThreads with {maxThreads}, {maxCPThreads}"); + opts.Initialize(loggerFactory); CreateMainStore(clusterFactory, out var checkpointDir); CreateObjectStore(clusterFactory, customCommandManager, checkpointDir, out var objectStoreSizeTracker); @@ -324,7 +325,7 @@ private void CreateObjectStore(IClusterFactory clusterFactory, CustomCommandMana objectStoreSizeTracker = null; if (!opts.DisableObjects) { - objKvSettings = opts.GetObjectStoreSettings(this.loggerFactory?.CreateLogger("TsavoriteKV [obj]"), + objKvSettings = opts.GetObjectStoreSettings(loggerFactory, out var objHeapMemorySize, out var objReadCacheHeapMemorySize); // Run checkpoint on its own thread to control p99 diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 399578e82d..0bcf20a7d7 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -4,6 +4,7 @@ using System; using System.Collections.Generic; using System.IO; +using System.Runtime; using Garnet.server.Auth.Settings; using Garnet.server.TLS; using Microsoft.Extensions.Logging; @@ -446,6 +447,16 @@ public class GarnetServerOptions : ServerOptions /// public UnixFileMode UnixSocketPermission { get; set; } + /// + /// Epoch instance used by server + /// + public LightEpoch Epoch; + + /// + /// Common state machine driver used by Garnet + /// + public StateMachineDriver StateMachineDriver; + /// /// Constructor /// @@ -454,6 +465,16 @@ public GarnetServerOptions(ILogger logger = null) : base(logger) this.logger = logger; } + /// + /// Initialize Garnet server options + /// + /// + public void Initialize(ILoggerFactory loggerFactory = null) + { + Epoch = new LightEpoch(); + StateMachineDriver = new(Epoch, loggerFactory?.CreateLogger($"StateMachineDriver")); + } + /// /// Get main store settings /// @@ -466,15 +487,15 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, if (MutablePercent is < 10 or > 95) throw new Exception("MutablePercent must be between 10 and 95"); - KVSettings kvSettings = new(baseDir: null, logger: logger); - var indexCacheLines = IndexSizeCachelines("hash index size", IndexSize); - kvSettings = new() + + KVSettings kvSettings = new() { IndexSize = indexCacheLines * 64L, PreallocateLog = false, MutableFraction = MutablePercent / 100.0, PageSize = 1L << PageSizeBits(), + Epoch = Epoch, loggerFactory = loggerFactory, logger = loggerFactory?.CreateLogger("TsavoriteKV [main]") }; @@ -618,23 +639,25 @@ public static int MemorySizeBits(string memorySize, string storePageSize, out in /// /// Get KVSettings for the object store log /// - public KVSettings GetObjectStoreSettings(ILogger logger, out long objHeapMemorySize, out long objReadCacheHeapMemorySize) + public KVSettings GetObjectStoreSettings(ILoggerFactory loggerFactory, out long objHeapMemorySize, out long objReadCacheHeapMemorySize) { objReadCacheHeapMemorySize = default; if (ObjectStoreMutablePercent is < 10 or > 95) throw new Exception("ObjectStoreMutablePercent must be between 10 and 95"); - KVSettings kvSettings = new(baseDir: null, logger: logger); - var indexCacheLines = IndexSizeCachelines("object store hash index size", ObjectStoreIndexSize); - kvSettings = new() + KVSettings kvSettings = new() { IndexSize = indexCacheLines * 64L, PreallocateLog = false, MutableFraction = ObjectStoreMutablePercent / 100.0, - PageSize = 1L << ObjectStorePageSizeBits() + PageSize = 1L << ObjectStorePageSizeBits(), + Epoch = Epoch, + loggerFactory = loggerFactory, + logger = loggerFactory?.CreateLogger("TsavoriteKV [obj]") }; + logger?.LogInformation("[Object Store] Using page size of {PageSize}", PrettySize(kvSettings.PageSize)); logger?.LogInformation("[Object Store] Each page can hold ~{PageSize} key-value pairs of objects", kvSettings.PageSize / 24); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs index ea05c8fc19..83293292c9 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineDriver.cs @@ -29,10 +29,10 @@ public class StateMachineDriver public SystemState SystemState => SystemState.Copy(ref systemState); - public StateMachineDriver(LightEpoch epoch, SystemState initialState, ILogger logger = null) + public StateMachineDriver(LightEpoch epoch, ILogger logger = null) { this.epoch = epoch; - this.systemState = initialState; + this.systemState = SystemState.Make(Phase.REST, 1); this.waitingList = []; this.logger = logger; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs index 43d5eab5a1..6192b71b7e 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs @@ -119,6 +119,11 @@ public sealed class KVSettings : IDisposable /// public RevivificationSettings RevivificationSettings; + /// + /// Epoch instance used by the store + /// + public LightEpoch Epoch = null; + /// /// Create default configuration settings for TsavoriteKV. You need to create and specify LogDevice /// explicitly with this API. diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index fcd0f5cb87..3f6e0eb8b2 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -87,7 +87,7 @@ internal void IncrementNumLockingSessions() /// Store-level user function implementations /// Func to call to create the allocator(s, if doing readcache) public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFunctions, Func allocatorFactory) - : base(kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger("TsavoriteKV Index Overflow buckets")) + : base(kvSettings.Epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger("TsavoriteKV Index Overflow buckets")) { this.allocatorFactory = allocatorFactory; loggerFactory = kvSettings.loggerFactory; @@ -158,7 +158,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun LockTable = new OverflowBucketLockTable(this); RevivificationManager = new(this, isFixedLenReviv, kvSettings.RevivificationSettings, logSettings); - stateMachineDriver = new(epoch, SystemState.Make(Phase.REST, 1), kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); + stateMachineDriver = new(epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); if (kvSettings.TryRecoverLatest) { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteBase.cs index 12efdb2850..d8ac3e5824 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteBase.cs @@ -52,9 +52,9 @@ public unsafe partial class TsavoriteBase /// /// Constructor /// - public TsavoriteBase(ILogger logger = null) + public TsavoriteBase(LightEpoch epoch = null, ILogger logger = null) { - epoch = new LightEpoch(); + this.epoch = epoch ?? new LightEpoch(); overflowBucketsAllocator = new MallocFixedPageSize(logger); } From 3071098530c98a4ddbaec4b40cd553b9fe6a2ce1 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 14:16:27 -0800 Subject: [PATCH 39/49] nit --- .../Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs index 203a8db06f..1e5996ae97 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/TsavoriteThread.cs @@ -1,7 +1,6 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT license. -using System.Collections.Generic; using System.Diagnostics; using System.Runtime.CompilerServices; using System.Threading; From f8a189962e9deedffb0032900da0e4dd21a0b570 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 17:38:28 -0800 Subject: [PATCH 40/49] add unified checkpointing logic to garnet --- libs/server/AOF/AofProcessor.cs | 5 +- libs/server/Resp/AdminCommands.cs | 4 +- libs/server/StoreWrapper.cs | 69 ++++++++++--------- .../core/Index/Checkpointing/Checkpoint.cs | 4 +- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 15 ++++ 5 files changed, 57 insertions(+), 40 deletions(-) diff --git a/libs/server/AOF/AofProcessor.cs b/libs/server/AOF/AofProcessor.cs index e3f8ee95c4..0472d93f73 100644 --- a/libs/server/AOF/AofProcessor.cs +++ b/libs/server/AOF/AofProcessor.cs @@ -210,11 +210,10 @@ public unsafe void ProcessAofRecordInternal(byte* ptr, int length, bool asReplic break; case AofEntryType.MainStoreCheckpointStartCommit: if (asReplica && header.storeVersion > storeWrapper.store.CurrentVersion) - _ = storeWrapper.TakeCheckpoint(false, StoreType.Main, logger); + _ = storeWrapper.TakeCheckpoint(false, logger); break; case AofEntryType.ObjectStoreCheckpointStartCommit: - if (asReplica && header.storeVersion > storeWrapper.objectStore.CurrentVersion) - _ = storeWrapper.TakeCheckpoint(false, StoreType.Object, logger); + // With unified checkpoint, we do not need to take object store checkpoint separately break; case AofEntryType.MainStoreCheckpointEndCommit: case AofEntryType.ObjectStoreCheckpointEndCommit: diff --git a/libs/server/Resp/AdminCommands.cs b/libs/server/Resp/AdminCommands.cs index 5905b61eee..676766a09a 100644 --- a/libs/server/Resp/AdminCommands.cs +++ b/libs/server/Resp/AdminCommands.cs @@ -818,7 +818,7 @@ private bool NetworkSAVE() return AbortWithWrongNumberOfArguments(nameof(RespCommand.SAVE)); } - if (!storeWrapper.TakeCheckpoint(false, StoreType.All, logger)) + if (!storeWrapper.TakeCheckpoint(false, logger)) { while (!RespWriteUtils.TryWriteError("ERR checkpoint already in progress"u8, ref dcurr, dend)) SendAndReset(); @@ -853,7 +853,7 @@ private bool NetworkBGSAVE() return AbortWithWrongNumberOfArguments(nameof(RespCommand.BGSAVE)); } - var success = storeWrapper.TakeCheckpoint(true, StoreType.All, logger); + var success = storeWrapper.TakeCheckpoint(true, logger); if (success) { while (!RespWriteUtils.TryWriteSimpleString("Background saving started"u8, ref dcurr, dend)) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 0def6ff075..60864cd319 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -761,28 +761,27 @@ public async Task TakeOnDemandCheckpoint(DateTimeOffset entryTime) } // Necessary to take a checkpoint because the latest checkpoint is before entryTime - await CheckpointTask(StoreType.All, logger: logger); + await CheckpointTask(logger); } /// /// Take checkpoint /// /// - /// /// /// - public bool TakeCheckpoint(bool background, StoreType storeType = StoreType.All, ILogger logger = null) + public bool TakeCheckpoint(bool background, ILogger logger = null) { // Prevent parallel checkpoint if (!TryPauseCheckpoints()) return false; if (background) - Task.Run(async () => await CheckpointTask(storeType, logger)); + Task.Run(async () => await CheckpointTask(logger)); else - CheckpointTask(storeType, logger).ConfigureAwait(false).GetAwaiter().GetResult(); + CheckpointTask(logger).ConfigureAwait(false).GetAwaiter().GetResult(); return true; } - private async Task CheckpointTask(StoreType storeType, ILogger logger = null) + private async Task CheckpointTask(ILogger logger = null) { try { @@ -803,12 +802,11 @@ private async Task CheckpointTask(StoreType storeType, ILogger logger = null) tryIncremental = false; var checkpointType = serverOptions.UseFoldOverCheckpoints ? CheckpointType.FoldOver : CheckpointType.Snapshot; - await InitiateCheckpoint(full, checkpointType, tryIncremental, storeType, logger); + await InitiateCheckpoint(full, checkpointType, tryIncremental, logger); if (full) { - if (storeType is StoreType.Main or StoreType.All) - this.lastSaveStoreTailAddress = lastSaveStoreTailAddress; - if (storeType is StoreType.Object or StoreType.All) + this.lastSaveStoreTailAddress = lastSaveStoreTailAddress; + if (objectStore != null) this.lastSaveObjectStoreTailAddress = lastSaveObjectStoreTailAddress; } lastSaveTime = DateTimeOffset.UtcNow; @@ -823,9 +821,9 @@ private async Task CheckpointTask(StoreType storeType, ILogger logger = null) } } - private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, bool tryIncremental, StoreType storeType, ILogger logger = null) + private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, bool tryIncremental, ILogger logger = null) { - logger?.LogInformation("Initiating checkpoint; full = {full}, type = {checkpointType}, tryIncremental = {tryIncremental}, storeType = {storeType}", full, checkpointType, tryIncremental, storeType); + logger?.LogInformation("Initiating checkpoint; full = {full}, type = {checkpointType}, tryIncremental = {tryIncremental}", full, checkpointType, tryIncremental); long CheckpointCoveredAofAddress = 0; if (appendOnlyFile != null) @@ -839,41 +837,46 @@ private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, logger?.LogInformation("Will truncate AOF to {tailAddress} after checkpoint (files deleted after next commit)", CheckpointCoveredAofAddress); } - (bool success, Guid token) storeCheckpointResult = default; - (bool success, Guid token) objectStoreCheckpointResult = default; + (bool success, Guid token) checkpointResult = default; + IStateMachine sm; if (full) { - if (storeType is StoreType.Main or StoreType.All) - { - storeCheckpointResult = await store.TakeFullCheckpointAsync(checkpointType); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); - } - - if (objectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) - { - objectStoreCheckpointResult = await objectStore.TakeFullCheckpointAsync(checkpointType); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, objectStore.CurrentVersion); - } + sm = objectStore == null ? + Checkpoint.Full(store, checkpointType, -1, out checkpointResult.token) : + Checkpoint.Full(store, objectStore, checkpointType, -1, out checkpointResult.token); } else { - if (storeType is StoreType.Main or StoreType.All) + tryIncremental &= store.CanTakeIncrementalCheckpoint(checkpointType, out checkpointResult.token); + if (objectStore != null) + tryIncremental &= objectStore.CanTakeIncrementalCheckpoint(checkpointType, out var guid2) & checkpointResult.token == guid2; + + if (tryIncremental) { - storeCheckpointResult = await store.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); + sm = objectStore == null ? + Checkpoint.IncrementalHybridLogOnly(store, -1, checkpointResult.token) : + Checkpoint.IncrementalHybridLogOnly(store, objectStore, -1, checkpointResult.token); } - - if (objectStore != null && (storeType == StoreType.Object || storeType == StoreType.All)) + else { - objectStoreCheckpointResult = await objectStore.TakeHybridLogCheckpointAsync(checkpointType, tryIncremental); - if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, objectStore.CurrentVersion); + sm = objectStore == null ? + Checkpoint.HybridLogOnly(store, checkpointType, -1, out checkpointResult.token) : + Checkpoint.HybridLogOnly(store, objectStore, checkpointType, -1, out checkpointResult.token); } } + checkpointResult.success = await serverOptions.StateMachineDriver.RunAsync(sm); + + if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) + { + EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); + EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, store.CurrentVersion); + } + // If cluster is enabled the replication manager is responsible for truncating AOF if (serverOptions.EnableCluster && serverOptions.EnableAOF) { - clusterProvider.SafeTruncateAOF(storeType, full, CheckpointCoveredAofAddress, storeCheckpointResult.token, objectStoreCheckpointResult.token); + clusterProvider.SafeTruncateAOF(objectStore == null ? StoreType.Main : StoreType.All, full, CheckpointCoveredAofAddress, checkpointResult.token, checkpointResult.token); } else { diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index 5f94cad543..ff0893b398 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -5,7 +5,7 @@ namespace Tsavorite.core { - static class Checkpoint + public static class Checkpoint { #region Single-store APIs public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) @@ -174,7 +174,7 @@ public static IStateMachine HybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, - CheckpointType checkpointType, long targetVersion, Guid guid) + long targetVersion, Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 3f6e0eb8b2..595969bfec 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -317,6 +317,21 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp return stateMachineDriver.Register(stateMachine); } + /// + /// Whether we can take an incremental snapshot checkpoint given current state of the store + /// + /// + /// + public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid guid) + { + guid = _lastSnapshotCheckpoint.info.guid; + return + checkpointType == CheckpointType.Snapshot + && guid != default + && _lastSnapshotCheckpoint.info.finalLogicalAddress > hlogBase.FlushedUntilAddress + && !hlog.HasObjectLog; + } + /// /// Take log-only checkpoint /// From b6df00cf43fd9fba86879251a1126b08ec9ead63 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 17:46:58 -0800 Subject: [PATCH 41/49] nit --- libs/server/Servers/GarnetServerOptions.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 0bcf20a7d7..2c49ba23ee 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -4,7 +4,6 @@ using System; using System.Collections.Generic; using System.IO; -using System.Runtime; using Garnet.server.Auth.Settings; using Garnet.server.TLS; using Microsoft.Extensions.Logging; From 5b4c646697243105d905595913064b9dfe826def Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 17:58:15 -0800 Subject: [PATCH 42/49] fix --- libs/server/Transaction/TransactionManager.cs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/libs/server/Transaction/TransactionManager.cs b/libs/server/Transaction/TransactionManager.cs index 0c131b06b1..5e9d0ead44 100644 --- a/libs/server/Transaction/TransactionManager.cs +++ b/libs/server/Transaction/TransactionManager.cs @@ -296,26 +296,28 @@ void BeginLockable(StoreType transactionStoreType) { while (true) { - if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Main) + if (transactionStoreType is StoreType.All or StoreType.Main) { lockableContext.BeginLockable(); } - if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Object) + if (transactionStoreType is StoreType.All or StoreType.Object) { if (objectStoreBasicContext.IsNull) throw new Exception("Trying to perform object store transaction with object store disabled"); if (objectStoreLockableContext.TryBeginLockable()) { // If we managed to begin lockable for the object store, we MUST be in the same version as the main store - return; + break; } objectStoreLockableContext.Refresh(); + if (transactionStoreType is StoreType.All or StoreType.Main) + { + lockableContext.EndLockable(); + lockableContext.Refresh(); + } + continue; } - if (transactionStoreType == StoreType.All || transactionStoreType == StoreType.Main) - { - lockableContext.EndLockable(); - lockableContext.Refresh(); - } + break; } } From 3e8c2c78aa70c69251c208a391968d3d44a37453 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Fri, 7 Mar 2025 18:32:05 -0800 Subject: [PATCH 43/49] use correct SMD --- libs/server/Servers/GarnetServerOptions.cs | 2 ++ .../storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs | 5 +++++ .../Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs | 2 +- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/libs/server/Servers/GarnetServerOptions.cs b/libs/server/Servers/GarnetServerOptions.cs index 2c49ba23ee..fd6a0d8f79 100644 --- a/libs/server/Servers/GarnetServerOptions.cs +++ b/libs/server/Servers/GarnetServerOptions.cs @@ -495,6 +495,7 @@ public KVSettings GetSettings(ILoggerFactory loggerFactory, MutableFraction = MutablePercent / 100.0, PageSize = 1L << PageSizeBits(), Epoch = Epoch, + StateMachineDriver = StateMachineDriver, loggerFactory = loggerFactory, logger = loggerFactory?.CreateLogger("TsavoriteKV [main]") }; @@ -653,6 +654,7 @@ public KVSettings GetObjectStoreSettings(ILoggerFactory l MutableFraction = ObjectStoreMutablePercent / 100.0, PageSize = 1L << ObjectStorePageSizeBits(), Epoch = Epoch, + StateMachineDriver = StateMachineDriver, loggerFactory = loggerFactory, logger = loggerFactory?.CreateLogger("TsavoriteKV [obj]") }; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs index 6192b71b7e..bfb6d9468a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Common/KVSettings.cs @@ -124,6 +124,11 @@ public sealed class KVSettings : IDisposable /// public LightEpoch Epoch = null; + /// + /// State machine driver for the store + /// + public StateMachineDriver StateMachineDriver = null; + /// /// Create default configuration settings for TsavoriteKV. You need to create and specify LogDevice /// explicitly with this API. diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 595969bfec..5a4333f7aa 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -158,7 +158,7 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun LockTable = new OverflowBucketLockTable(this); RevivificationManager = new(this, isFixedLenReviv, kvSettings.RevivificationSettings, logSettings); - stateMachineDriver = new(epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); + stateMachineDriver = kvSettings.StateMachineDriver ?? new(epoch, kvSettings.logger ?? kvSettings.loggerFactory?.CreateLogger($"StateMachineDriver")); if (kvSettings.TryRecoverLatest) { From 576f92d7ab5796e52c20795c0c491736e83efef8 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 12:30:11 -0800 Subject: [PATCH 44/49] nit --- libs/server/StoreWrapper.cs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 60864cd319..4ade88feca 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -870,7 +870,8 @@ private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, if (serverOptions.EnableCluster && clusterProvider.IsPrimary()) { EnqueueCommit(AofEntryType.MainStoreCheckpointEndCommit, store.CurrentVersion); - EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, store.CurrentVersion); + if (objectStore != null) + EnqueueCommit(AofEntryType.ObjectStoreCheckpointEndCommit, objectStore.CurrentVersion); } // If cluster is enabled the replication manager is responsible for truncating AOF From 76430c19b21f594bb4cfcb9160cec84abeea5cab Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 12:54:12 -0800 Subject: [PATCH 45/49] fix --- libs/server/StoreWrapper.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 4ade88feca..7a2d29b78f 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -877,7 +877,7 @@ private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, // If cluster is enabled the replication manager is responsible for truncating AOF if (serverOptions.EnableCluster && serverOptions.EnableAOF) { - clusterProvider.SafeTruncateAOF(objectStore == null ? StoreType.Main : StoreType.All, full, CheckpointCoveredAofAddress, checkpointResult.token, checkpointResult.token); + clusterProvider.SafeTruncateAOF(StoreType.All, full, CheckpointCoveredAofAddress, checkpointResult.token, checkpointResult.token); } else { From 7b7bec5d9e1c54fcbdb6e982275405b4d6efd500 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 13:26:38 -0800 Subject: [PATCH 46/49] nit --- .../Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index b4d99098c6..81f6381c40 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -31,6 +31,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri switch (next.Phase) { case Phase.PREPARE: + base.GlobalBeforeEnteringState(next, stateMachineDriver); store._lastSnapshotCheckpoint.Dispose(); store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); @@ -40,6 +41,7 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri break; case Phase.WAIT_FLUSH: + base.GlobalBeforeEnteringState(next, stateMachineDriver); var finalLogicalAddress = store.hlogBase.GetTailAddress(); store.StreamingSnapshotScanPhase2(finalLogicalAddress); break; From bbad4f0668f2c7c0ddd8d1706804a6469e7ed0ce Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 13:37:28 -0800 Subject: [PATCH 47/49] fix test as versions are different due to unified ckpt --- .../ClusterReplicationDisklessSyncTests.cs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index a8075f0716..325f74d57a 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -236,7 +236,9 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool 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); + + var expectedVersion1 = disableObjects ? 2 : 3; + ClassicAssert.AreEqual(expectedVersion1, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); // Reset and re-attach replica as primary @@ -258,9 +260,10 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, 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); + var expectedVersion2 = disableObjects ? 3 : 5; + ClassicAssert.AreEqual(expectedVersion2, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); - ClassicAssert.AreEqual(2, replicaOneVersion); + ClassicAssert.AreEqual(expectedVersion1, replicaOneVersion); // Re-attach first replica _ = context.clusterTestUtils.ClusterReplicate(replicaNodeIndex: replicaOneIndex, primaryNodeIndex: primaryIndex, logger: context.logger); @@ -271,7 +274,8 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool 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); + var expectedVersion3 = disableObjects ? 4 : 7; + ClassicAssert.AreEqual(expectedVersion3, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); } From af870fb9a680e98ec253a87aa65f76473129d294 Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 14:22:03 -0800 Subject: [PATCH 48/49] add comment --- .../Checkpointing/StreamingSnapshotCheckpointSMTask.cs | 1 - .../ClusterReplicationDisklessSyncTests.cs | 8 ++++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index 81f6381c40..79cc432012 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -35,7 +35,6 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._lastSnapshotCheckpoint.Dispose(); store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - store._hybridLogCheckpoint.info.version = next.Version; store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; store.StreamingSnapshotScanPhase1(); break; diff --git a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs index 325f74d57a..389fa5d55d 100644 --- a/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs +++ b/test/Garnet.test.cluster/ReplicationTests/ClusterReplicationDisklessSyncTests.cs @@ -237,6 +237,8 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, var primaryVersion = context.clusterTestUtils.GetStoreCurrentVersion(primaryIndex, isMainStore: true, logger: context.logger); var replicaOneVersion = context.clusterTestUtils.GetStoreCurrentVersion(replicaOneIndex, isMainStore: true, logger: context.logger); + // With unified store, versions increase per scan (main and object) + // so expected versions depend on whether objects are disabled or not var expectedVersion1 = disableObjects ? 2 : 3; ClassicAssert.AreEqual(expectedVersion1, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); @@ -260,6 +262,9 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool disableObjects, 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); + + // With unified store, versions increase per scan (main and object) + // so expected versions depend on whether objects are disabled or not var expectedVersion2 = disableObjects ? 3 : 5; ClassicAssert.AreEqual(expectedVersion2, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaTwoVersion); @@ -274,6 +279,9 @@ public void ClusterDBVersionAlignmentDisklessSync([Values] bool 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); + + // With unified store, versions increase per scan (main and object) + // so expected versions depend on whether objects are disabled or not var expectedVersion3 = disableObjects ? 4 : 7; ClassicAssert.AreEqual(expectedVersion3, primaryVersion); ClassicAssert.AreEqual(primaryVersion, replicaOneVersion); From 2bccee7a656ce16e4ae89a2589a77838f369ff9f Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 8 Mar 2025 17:53:40 -0800 Subject: [PATCH 49/49] remove targetVersion from checkpoint API, versions always progress by 1. --- libs/server/StoreWrapper.cs | 12 ++-- .../IStreamingSnapshotIteratorFunctions.cs | 4 +- .../core/Index/Checkpointing/Checkpoint.cs | 72 +++++++++---------- .../Index/Checkpointing/FullCheckpointSM.cs | 8 +-- .../Checkpointing/HybridLogCheckpointSM.cs | 5 +- .../Index/Checkpointing/IndexCheckpointSM.cs | 4 +- .../core/Index/Checkpointing/IndexResizeSM.cs | 2 +- .../Index/Checkpointing/StateMachineBase.cs | 5 +- .../StreamingSnapshotCheckpointSM.cs | 7 +- .../StreamingSnapshotCheckpointSMTask.cs | 13 ++-- .../StreamingSnapshotTsavoriteKV.cs | 8 +-- .../Index/Checkpointing/VersionChangeSM.cs | 6 +- .../cs/src/core/Index/Tsavorite/Tsavorite.cs | 46 ++++-------- .../Tsavorite/cs/test/RecoveryChecks.cs | 4 +- .../cs/test/StateMachineBarrierTests.cs | 5 +- 15 files changed, 87 insertions(+), 114 deletions(-) diff --git a/libs/server/StoreWrapper.cs b/libs/server/StoreWrapper.cs index 7a2d29b78f..555fcec299 100644 --- a/libs/server/StoreWrapper.cs +++ b/libs/server/StoreWrapper.cs @@ -842,8 +842,8 @@ private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, if (full) { sm = objectStore == null ? - Checkpoint.Full(store, checkpointType, -1, out checkpointResult.token) : - Checkpoint.Full(store, objectStore, checkpointType, -1, out checkpointResult.token); + Checkpoint.Full(store, checkpointType, out checkpointResult.token) : + Checkpoint.Full(store, objectStore, checkpointType, out checkpointResult.token); } else { @@ -854,14 +854,14 @@ private async Task InitiateCheckpoint(bool full, CheckpointType checkpointType, if (tryIncremental) { sm = objectStore == null ? - Checkpoint.IncrementalHybridLogOnly(store, -1, checkpointResult.token) : - Checkpoint.IncrementalHybridLogOnly(store, objectStore, -1, checkpointResult.token); + Checkpoint.IncrementalHybridLogOnly(store, checkpointResult.token) : + Checkpoint.IncrementalHybridLogOnly(store, objectStore, checkpointResult.token); } else { sm = objectStore == null ? - Checkpoint.HybridLogOnly(store, checkpointType, -1, out checkpointResult.token) : - Checkpoint.HybridLogOnly(store, objectStore, checkpointType, -1, out checkpointResult.token); + Checkpoint.HybridLogOnly(store, checkpointType, out checkpointResult.token) : + Checkpoint.HybridLogOnly(store, objectStore, checkpointType, out checkpointResult.token); } } diff --git a/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs b/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs index cd95e696be..422353aa74 100644 --- a/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs +++ b/libs/storage/Tsavorite/cs/src/core/Allocator/IStreamingSnapshotIteratorFunctions.cs @@ -13,9 +13,9 @@ public interface IStreamingSnapshotIteratorFunctions /// Iteration is starting. /// Checkpoint token /// Current version of database - /// Target version of database + /// Next version of database /// True to continue iteration, else false - bool OnStart(Guid checkpointToken, long currentVersion, long targetVersion); + bool OnStart(Guid checkpointToken, long currentVersion, long nextVersion); /// Next record in the streaming snapshot. /// Reference to the current record's key diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs index ff0893b398..1295be2461 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/Checkpoint.cs @@ -8,7 +8,7 @@ namespace Tsavorite.core public static class Checkpoint { #region Single-store APIs - public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) + public static IStateMachine Full(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { @@ -18,12 +18,12 @@ public static IStateMachine Full(Tsav if (checkpointType == CheckpointType.FoldOver) { var backend = new FoldOverSMTask(store, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask, backend); + return new FullCheckpointSM(indexCheckpointTask, backend); } else if (checkpointType == CheckpointType.Snapshot) { var backend = new SnapshotCheckpointSMTask(store, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask, backend); + return new FullCheckpointSM(indexCheckpointTask, backend); } else { @@ -31,31 +31,25 @@ public static IStateMachine Full(Tsav } } - public static IStateMachine Streaming( - TsavoriteKV store1, - TsavoriteKV store2, - long targetVersion, out Guid guid) - where TStoreFunctions1 : IStoreFunctions - where TAllocator1 : IAllocator - where TStoreFunctions2 : IStoreFunctions - where TAllocator2 : IAllocator + public static IStateMachine Streaming(TsavoriteKV store, out Guid guid) + where TStoreFunctions : IStoreFunctions + where TAllocator : IAllocator { guid = Guid.NewGuid(); - var backend1 = new StreamingSnapshotCheckpointSMTask(targetVersion, store1, guid); - var backend2 = new StreamingSnapshotCheckpointSMTask(targetVersion, store2, guid); - return new StreamingSnapshotCheckpointSM(targetVersion, backend1, backend2); + var backend = new StreamingSnapshotCheckpointSMTask(store, guid); + return new StreamingSnapshotCheckpointSM(backend); } - public static IStateMachine IndexOnly(TsavoriteKV store, long targetVersion, out Guid guid) + public static IStateMachine IndexOnly(TsavoriteKV store, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { guid = Guid.NewGuid(); var indexCheckpointTask = new IndexCheckpointSMTask(store, guid); - return new IndexCheckpointSM(targetVersion, indexCheckpointTask); + return new IndexCheckpointSM(indexCheckpointTask); } - public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, long targetVersion, out Guid guid) + public static IStateMachine HybridLogOnly(TsavoriteKV store, CheckpointType checkpointType, out Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { @@ -64,12 +58,12 @@ public static IStateMachine HybridLogOnly(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); + return new HybridLogCheckpointSM(backend); } else if (checkpointType == CheckpointType.Snapshot) { var backend = new SnapshotCheckpointSMTask(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); + return new HybridLogCheckpointSM(backend); } else { @@ -77,12 +71,12 @@ public static IStateMachine HybridLogOnly(TsavoriteKV store, long targetVersion, Guid guid) + public static IStateMachine IncrementalHybridLogOnly(TsavoriteKV store, Guid guid) where TStoreFunctions : IStoreFunctions where TAllocator : IAllocator { var backend = new IncrementalSnapshotCheckpointSMTask(store, guid); - return new HybridLogCheckpointSM(targetVersion, backend); + return new HybridLogCheckpointSM(backend); } #endregion @@ -90,7 +84,7 @@ public static IStateMachine IncrementalHybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, - CheckpointType checkpointType, long targetVersion, out Guid guid) + CheckpointType checkpointType, out Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions @@ -104,13 +98,13 @@ public static IStateMachine Full(store1, guid); var backend2 = new FoldOverSMTask(store2, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + return new FullCheckpointSM(indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); } else if (checkpointType == CheckpointType.Snapshot) { var backend1 = new SnapshotCheckpointSMTask(store1, guid); var backend2 = new SnapshotCheckpointSMTask(store2, guid); - return new FullCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); + return new FullCheckpointSM(indexCheckpointTask1, indexCheckpointTask2, backend1, backend2); } else { @@ -118,19 +112,25 @@ public static IStateMachine Full(TsavoriteKV store, long targetVersion, out Guid guid) - where TStoreFunctions : IStoreFunctions - where TAllocator : IAllocator + public static IStateMachine Streaming( + TsavoriteKV store1, + TsavoriteKV store2, + out Guid guid) + where TStoreFunctions1 : IStoreFunctions + where TAllocator1 : IAllocator + where TStoreFunctions2 : IStoreFunctions + where TAllocator2 : IAllocator { guid = Guid.NewGuid(); - var backend = new StreamingSnapshotCheckpointSMTask(targetVersion, store, guid); - return new StreamingSnapshotCheckpointSM(targetVersion, backend); + var backend1 = new StreamingSnapshotCheckpointSMTask(store1, guid); + var backend2 = new StreamingSnapshotCheckpointSMTask(store2, guid); + return new StreamingSnapshotCheckpointSM(backend1, backend2); } public static IStateMachine IndexOnly( TsavoriteKV store1, TsavoriteKV store2, - long targetVersion, out Guid guid) + out Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions @@ -139,13 +139,13 @@ public static IStateMachine IndexOnly(store1, guid); var indexCheckpointTask2 = new IndexCheckpointSMTask(store2, guid); - return new IndexCheckpointSM(targetVersion, indexCheckpointTask1, indexCheckpointTask2); + return new IndexCheckpointSM(indexCheckpointTask1, indexCheckpointTask2); } public static IStateMachine HybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, - CheckpointType checkpointType, long targetVersion, out Guid guid) + CheckpointType checkpointType, out Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions @@ -157,13 +157,13 @@ public static IStateMachine HybridLogOnly(store1, guid); var backend2 = new FoldOverSMTask(store2, guid); - return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + return new HybridLogCheckpointSM(backend1, backend2); } else if (checkpointType == CheckpointType.Snapshot) { var backend1 = new SnapshotCheckpointSMTask(store1, guid); var backend2 = new SnapshotCheckpointSMTask(store2, guid); - return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + return new HybridLogCheckpointSM(backend1, backend2); } else { @@ -174,7 +174,7 @@ public static IStateMachine HybridLogOnly( TsavoriteKV store1, TsavoriteKV store2, - long targetVersion, Guid guid) + Guid guid) where TStoreFunctions1 : IStoreFunctions where TAllocator1 : IAllocator where TStoreFunctions2 : IStoreFunctions @@ -182,7 +182,7 @@ public static IStateMachine IncrementalHybridLogOnly(store1, guid); var backend2 = new IncrementalSnapshotCheckpointSMTask(store2, guid); - return new HybridLogCheckpointSM(targetVersion, backend1, backend2); + return new HybridLogCheckpointSM(backend1, backend2); } #endregion } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs index f35f63373c..8b1889714f 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/FullCheckpointSM.cs @@ -9,13 +9,11 @@ namespace Tsavorite.core internal sealed class FullCheckpointSM : HybridLogCheckpointSM { /// - /// Construct a new FullCheckpointStateMachine to use the given set of checkpoint tasks, - /// drawing boundary at targetVersion. + /// Construct a new FullCheckpointStateMachine to use the given set of checkpoint tasks. /// - /// upper limit (inclusive) of the version included /// Tasks - public FullCheckpointSM(long targetVersion = -1, params IStateMachineTask[] tasks) - : base(targetVersion, tasks) + public FullCheckpointSM(params IStateMachineTask[] tasks) + : base(tasks) { } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs index 501f79152b..76dd21734d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/HybridLogCheckpointSM.cs @@ -11,10 +11,9 @@ internal class HybridLogCheckpointSM : VersionChangeSM /// /// Construct a new HybridLogCheckpointStateMachine with the given tasks. Does not load any tasks by default. /// - /// upper limit (inclusive) of the version included /// The tasks to load onto the state machine - public HybridLogCheckpointSM(long targetVersion, params IStateMachineTask[] tasks) - : base(targetVersion, tasks) { } + public HybridLogCheckpointSM(params IStateMachineTask[] tasks) + : base(tasks) { } /// public override SystemState NextState(SystemState start) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs index 1a300e9bbf..4e1f46a8bf 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexCheckpointSM.cs @@ -11,8 +11,8 @@ internal sealed class IndexCheckpointSM : StateMachineBase /// /// Create a new IndexSnapshotStateMachine /// - public IndexCheckpointSM(long targetVersion = -1, params IStateMachineTask[] tasks) - : base(targetVersion, tasks) + public IndexCheckpointSM(params IStateMachineTask[] tasks) + : base(tasks) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs index 0bff11bdcd..d15856d311 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/IndexResizeSM.cs @@ -11,7 +11,7 @@ internal sealed class IndexResizeSM : StateMachineBase /// /// Constructs a new IndexResizeStateMachine /// - public IndexResizeSM(int targetVersion = -1, params IStateMachineTask[] tasks) : base(targetVersion, tasks) + public IndexResizeSM(params IStateMachineTask[] tasks) : base(tasks) { } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs index 7b0add4b56..f031e40f8a 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StateMachineBase.cs @@ -10,17 +10,14 @@ namespace Tsavorite.core internal abstract class StateMachineBase : IStateMachine { readonly IStateMachineTask[] tasks; - protected long toVersion; /// /// Construct a new SynchronizationStateMachine with the given tasks. The order of tasks given is the /// order they are executed on each state machine. /// - /// To version /// The ISynchronizationTasks to run on the state machine - protected StateMachineBase(long toVersion = -1, params IStateMachineTask[] tasks) + protected StateMachineBase(params IStateMachineTask[] tasks) { - this.toVersion = toVersion; this.tasks = tasks; } diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs index cf18ba1346..bc0d401e76 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSM.cs @@ -9,11 +9,10 @@ namespace Tsavorite.core class StreamingSnapshotCheckpointSM : VersionChangeSM { /// - /// Construct a new StreamingSnapshotCheckpointStateMachine, drawing boundary at targetVersion. + /// Construct a new StreamingSnapshotCheckpointStateMachine. /// - /// upper limit (inclusive) of the version included - public StreamingSnapshotCheckpointSM(long targetVersion, params IStateMachineTask[] tasks) - : base(targetVersion, tasks) + public StreamingSnapshotCheckpointSM(params IStateMachineTask[] tasks) + : base(tasks) { } /// diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs index 79cc432012..f2a5ca871d 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotCheckpointSMTask.cs @@ -17,12 +17,9 @@ sealed class StreamingSnapshotCheckpointSMTask where TAllocator : IAllocator { - readonly long targetVersion; - - public StreamingSnapshotCheckpointSMTask(long targetVersion, TsavoriteKV store, Guid guid) + public StreamingSnapshotCheckpointSMTask(TsavoriteKV store, Guid guid) : base(store, guid) { - this.targetVersion = targetVersion; } /// @@ -35,9 +32,15 @@ public override void GlobalBeforeEnteringState(SystemState next, StateMachineDri store._lastSnapshotCheckpoint.Dispose(); store._hybridLogCheckpointToken = guid; store.InitializeHybridLogCheckpoint(store._hybridLogCheckpointToken, next.Version); - store._hybridLogCheckpoint.info.nextVersion = targetVersion == -1 ? next.Version + 1 : targetVersion; + store._hybridLogCheckpoint.info.nextVersion = next.Version + 1; store.StreamingSnapshotScanPhase1(); break; + + case Phase.IN_PROGRESS: + if (store._hybridLogCheckpoint.info.nextVersion != next.Version) + throw new TsavoriteException($"StreamingSnapshotCheckpointSMTask: IN_PROGRESS phase with incorrect version {next.Version}, expected {store._hybridLogCheckpoint.info.nextVersion}"); + base.GlobalBeforeEnteringState(next, stateMachineDriver); + break; case Phase.WAIT_FLUSH: base.GlobalBeforeEnteringState(next, stateMachineDriver); diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs index 56694ed8ec..a6dcda90a8 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/StreamingSnapshotTsavoriteKV.cs @@ -24,15 +24,15 @@ class ScanPhase1Functions : IScanIteratorFunctions readonly IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions; readonly Guid checkpointToken; readonly long currentVersion; - readonly long targetVersion; + readonly long nextVersion; public long numberOfRecords; - public ScanPhase1Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, Guid checkpointToken, long currentVersion, long targetVersion) + public ScanPhase1Functions(IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions, Guid checkpointToken, long currentVersion, long nextVersion) { this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; this.checkpointToken = checkpointToken; this.currentVersion = currentVersion; - this.targetVersion = targetVersion; + this.nextVersion = nextVersion; } /// @@ -52,7 +52,7 @@ public void OnException(Exception exception, long numberOfRecords) /// public bool OnStart(long beginAddress, long endAddress) - => streamingSnapshotIteratorFunctions.OnStart(checkpointToken, currentVersion, targetVersion); + => streamingSnapshotIteratorFunctions.OnStart(checkpointToken, currentVersion, nextVersion); /// public void OnStop(bool completed, long numberOfRecords) diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs index 7197760f5b..67802d0f8b 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Checkpointing/VersionChangeSM.cs @@ -11,9 +11,8 @@ internal class VersionChangeSM : StateMachineBase /// /// Construct a new VersionChangeStateMachine with the given tasks. Does not load any tasks by default. /// - /// upper limit (inclusive) of the version included /// The tasks to load onto the state machine - protected VersionChangeSM(long targetVersion = -1, params IStateMachineTask[] tasks) : base(targetVersion, tasks) + protected VersionChangeSM(params IStateMachineTask[] tasks) : base(tasks) { } @@ -28,8 +27,7 @@ public override SystemState NextState(SystemState start) break; case Phase.PREPARE: nextState.Phase = Phase.IN_PROGRESS; - if (toVersion == -1) toVersion = start.Version + 1; - nextState.Version = toVersion; + nextState.Version = start.Version + 1; break; case Phase.IN_PROGRESS: nextState.Phase = Phase.REST; diff --git a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs index 5a4333f7aa..d89a621798 100644 --- a/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs +++ b/libs/storage/Tsavorite/cs/src/core/Index/Tsavorite/Tsavorite.cs @@ -181,18 +181,13 @@ public TsavoriteKV(KVSettings kvSettings, TStoreFunctions storeFun /// /// Checkpoint token /// Checkpoint type - /// - /// intended version number of the next version. Checkpoint will not execute if supplied version is not larger - /// than current version. Actual new version may have version number greater than supplied number. If the supplied - /// number is -1, checkpoint will unconditionally create a new version. - /// /// Iterator for streaming snapshot records /// /// Whether we successfully initiated the checkpoint (initiation may /// fail if we are already taking a checkpoint or performing some other /// operation such as growing the index). Use CompleteCheckpointAsync to wait completion. /// - public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointType, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { IStateMachine stateMachine; @@ -201,11 +196,11 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT if (streamingSnapshotIteratorFunctions is null) throw new TsavoriteException("StreamingSnapshot checkpoint requires a streaming snapshot iterator"); this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; - stateMachine = Checkpoint.Streaming(this, targetVersion, out token); + stateMachine = Checkpoint.Streaming(this, out token); } else { - stateMachine = Checkpoint.Full(this, checkpointType, targetVersion, out token); + stateMachine = Checkpoint.Full(this, checkpointType, out token); } return stateMachineDriver.Register(stateMachine); } @@ -215,11 +210,6 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// /// Checkpoint type /// Cancellation token - /// - /// intended version number of the next version. Checkpoint will not execute if supplied version is not larger - /// than current version. Actual new version may have version number greater than supplied number. If the supplied - /// number is -1, checkpoint will unconditionally create a new version. - /// /// Iterator for streaming snapshot records /// /// (bool success, Guid token) @@ -230,9 +220,9 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// Await task to complete checkpoint, if initiated successfully /// public async ValueTask<(bool success, Guid token)> TakeFullCheckpointAsync(CheckpointType checkpointType, - CancellationToken cancellationToken = default, long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + CancellationToken cancellationToken = default, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { - var success = TryInitiateFullCheckpoint(out Guid token, checkpointType, targetVersion, streamingSnapshotIteratorFunctions); + var success = TryInitiateFullCheckpoint(out Guid token, checkpointType, streamingSnapshotIteratorFunctions); if (success) await CompleteCheckpointAsync(cancellationToken).ConfigureAwait(false); @@ -247,7 +237,7 @@ public bool TryInitiateFullCheckpoint(out Guid token, CheckpointType checkpointT /// Whether we could initiate the checkpoint. Use CompleteCheckpointAsync to wait completion. public bool TryInitiateIndexCheckpoint(out Guid token) { - var stateMachine = Checkpoint.IndexOnly(this, -1, out token); + var stateMachine = Checkpoint.IndexOnly(this, out token); return stateMachineDriver.Register(stateMachine); } @@ -279,14 +269,9 @@ public bool TryInitiateIndexCheckpoint(out Guid token) /// Checkpoint token /// Checkpoint type /// For snapshot, try to store as incremental delta over last snapshot - /// - /// intended version number of the next version. Checkpoint will not execute if supplied version is not larger - /// than current version. Actual new version may have version number greater than supplied number. If the supplied - /// number is -1, checkpoint will unconditionally create a new version. - /// /// Whether we could initiate the checkpoint. Use CompleteCheckpointAsync to wait completion. public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkpointType, bool tryIncremental = false, - long targetVersion = -1, IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) + IStreamingSnapshotIteratorFunctions streamingSnapshotIteratorFunctions = null) { IStateMachine stateMachine; @@ -295,7 +280,7 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp if (streamingSnapshotIteratorFunctions is null) throw new TsavoriteException("StreamingSnapshot checkpoint requires a streaming snapshot iterator"); this.streamingSnapshotIteratorFunctions = streamingSnapshotIteratorFunctions; - stateMachine = Checkpoint.Streaming(this, targetVersion, out token); + stateMachine = Checkpoint.Streaming(this, out token); } else { @@ -307,11 +292,11 @@ public bool TryInitiateHybridLogCheckpoint(out Guid token, CheckpointType checkp && !hlog.HasObjectLog; if (incremental) { - stateMachine = Checkpoint.IncrementalHybridLogOnly(this, targetVersion, token); + stateMachine = Checkpoint.IncrementalHybridLogOnly(this, token); } else { - stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, targetVersion, out token); + stateMachine = Checkpoint.HybridLogOnly(this, checkpointType, out token); } } return stateMachineDriver.Register(stateMachine); @@ -338,11 +323,6 @@ public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid /// Checkpoint type /// For snapshot, try to store as incremental delta over last snapshot /// Cancellation token - /// - /// intended version number of the next version. Checkpoint will not execute if supplied version is not larger - /// than current version. Actual new version may have version number greater than supplied number. If the supplied - /// number is -1, checkpoint will unconditionally create a new version. - /// /// /// (bool success, Guid token) /// success: Whether we successfully initiated the checkpoint (initiation may @@ -352,9 +332,9 @@ public bool CanTakeIncrementalCheckpoint(CheckpointType checkpointType, out Guid /// Await task to complete checkpoint, if initiated successfully /// public async ValueTask<(bool success, Guid token)> TakeHybridLogCheckpointAsync(CheckpointType checkpointType, - bool tryIncremental = false, CancellationToken cancellationToken = default, long targetVersion = -1) + bool tryIncremental = false, CancellationToken cancellationToken = default) { - var success = TryInitiateHybridLogCheckpoint(out Guid token, checkpointType, tryIncremental, targetVersion); + var success = TryInitiateHybridLogCheckpoint(out Guid token, checkpointType, tryIncremental); if (success) await CompleteCheckpointAsync(cancellationToken).ConfigureAwait(false); @@ -600,7 +580,7 @@ public async Task GrowIndexAsync() throw new TsavoriteException("Cannot use GrowIndex when using non-async sessions"); var indexResizeTask = new IndexResizeSMTask(this); - var indexResizeSM = new IndexResizeSM(-1, indexResizeTask); + var indexResizeSM = new IndexResizeSM(indexResizeTask); return await stateMachineDriver.RunAsync(indexResizeSM); } diff --git a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs index 4df0676d56..a13d60f9d9 100644 --- a/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs +++ b/libs/storage/Tsavorite/cs/test/RecoveryChecks.cs @@ -958,9 +958,9 @@ public SnapshotIterator(TsavoriteKV(new MyFunctions()); bc2 = session2.BasicContext; return true; diff --git a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs index 13e695812c..7ccd56b414 100644 --- a/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs +++ b/libs/storage/Tsavorite/cs/test/StateMachineBarrierTests.cs @@ -101,8 +101,7 @@ public void StateMachineBarrierTest1() void Prepare(out SMSimpleFunctions f, out ClientSession s1, out UnsafeContext uc1, - out ThreadSession s2, - long toVersion = -1) + out ThreadSession s2) { f = new SMSimpleFunctions(); @@ -141,7 +140,7 @@ void Prepare(out SMSimpleFunctions f, // We should be in REST, 1 ClassicAssert.IsTrue(SystemState.Equal(SystemState.Make(Phase.REST, 1), store.SystemState)); - _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver, targetVersion: toVersion); + _ = store.TryInitiateHybridLogCheckpoint(out _, CheckpointType.FoldOver); // Wait for PREPARE phase while (!SystemState.Equal(SystemState.Make(Phase.PREPARE, 1), store.SystemState))