diff --git a/cs/playground/FasterKVDiskReadBenchmark/Program.cs b/cs/playground/FasterKVDiskReadBenchmark/Program.cs index d05068b26..0af0b1bdc 100644 --- a/cs/playground/FasterKVDiskReadBenchmark/Program.cs +++ b/cs/playground/FasterKVDiskReadBenchmark/Program.cs @@ -117,7 +117,7 @@ static async Task AsyncReadOperator(int id) Input input = default; int i = 0; - var tasks = new (long, ValueTask.ReadAsyncResult>)[batchSize]; + var tasks = new (long, ValueTask.ReadAsyncResult>)[batchSize]; while (true) { key = new Key(NumKeys * id + rand.Next(0, NumKeys)); diff --git a/cs/playground/VarLenStructSample/Types.cs b/cs/playground/VarLenStructSample/Types.cs index de4e2185b..8cba2dd23 100644 --- a/cs/playground/VarLenStructSample/Types.cs +++ b/cs/playground/VarLenStructSample/Types.cs @@ -77,5 +77,10 @@ public int GetLength(ref VarLenType t) { return sizeof(int) * t.length; } + + public int GetLength(ref VarLenType t, ref Input input) + { + return sizeof(int) * t.length; + } } } diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index f6b0a130a..15fada251 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -293,6 +293,13 @@ public unsafe abstract partial class AllocatorBase : IDisposable /// public abstract int GetRecordSize(long physicalAddress); + /// + /// Get record size + /// + /// + /// + /// + public abstract int GetRecordSize(long physicalAddress, ref Input input); /// /// Get number of bytes required diff --git a/cs/src/core/Allocator/BlittableAllocator.cs b/cs/src/core/Allocator/BlittableAllocator.cs index 33fdf930f..5da3efb39 100644 --- a/cs/src/core/Allocator/BlittableAllocator.cs +++ b/cs/src/core/Allocator/BlittableAllocator.cs @@ -67,6 +67,11 @@ public override int GetRecordSize(long physicalAddress) return recordSize; } + public override int GetRecordSize(long physicalAddress, ref Input input) + { + return recordSize; + } + public override int GetAverageRecordSize() { return recordSize; diff --git a/cs/src/core/Allocator/GenericAllocator.cs b/cs/src/core/Allocator/GenericAllocator.cs index 97b53e27c..ed087e951 100644 --- a/cs/src/core/Allocator/GenericAllocator.cs +++ b/cs/src/core/Allocator/GenericAllocator.cs @@ -114,7 +114,6 @@ public override ref RecordInfo GetInfoFromBytePointer(byte* ptr) return ref Unsafe.AsRef>(ptr).info; } - public override ref Key GetKey(long physicalAddress) { // Offset within page @@ -142,6 +141,11 @@ public override int GetRecordSize(long physicalAddress) return recordSize; } + public override int GetRecordSize(long physicalAddress, ref Input input) + { + return recordSize; + } + public override int GetAverageRecordSize() { return recordSize; diff --git a/cs/src/core/Allocator/VarLenBlittableAllocator.cs b/cs/src/core/Allocator/VarLenBlittableAllocator.cs index c180d3717..89e290a9f 100644 --- a/cs/src/core/Allocator/VarLenBlittableAllocator.cs +++ b/cs/src/core/Allocator/VarLenBlittableAllocator.cs @@ -89,6 +89,11 @@ private int ValueSize(long physicalAddress) return ValueLength.GetLength(ref GetValue(physicalAddress)); } + private int ValueSize(long physicalAddress, ref Input input) + { + return ValueLength.GetLength(ref GetValue(physicalAddress), ref input); + } + public override int GetRecordSize(long physicalAddress) { ref var recordInfo = ref GetInfo(physicalAddress); @@ -100,6 +105,17 @@ public override int GetRecordSize(long physicalAddress) return size; } + public override int GetRecordSize(long physicalAddress, ref Input input) + { + ref var recordInfo = ref GetInfo(physicalAddress); + if (recordInfo.IsNull()) + return RecordInfo.GetLength(); + + var size = RecordInfo.GetLength() + KeySize(physicalAddress) + ValueSize(physicalAddress, ref input); + size = (size + kRecordAlignment - 1) & (~(kRecordAlignment - 1)); + return size; + } + public override int GetRequiredRecordSize(long physicalAddress, int availableBytes) { // We need at least [record size] + [average key size] + [average value size] diff --git a/cs/src/core/ClientSession/ClientSession.cs b/cs/src/core/ClientSession/ClientSession.cs index 4e66f5433..f25e379a0 100644 --- a/cs/src/core/ClientSession/ClientSession.cs +++ b/cs/src/core/ClientSession/ClientSession.cs @@ -20,25 +20,34 @@ namespace FASTER.core /// /// /// - public sealed class ClientSession : IDisposable + public sealed class ClientSession : IClientSession, IDisposable where Key : new() where Value : new() where Functions : IFunctions { + private readonly FasterKV fht; + internal readonly bool SupportAsync = false; - private readonly FasterKV fht; - internal readonly FasterKV.FasterExecutionContext ctx; + internal readonly FasterKV.FasterExecutionContext ctx; internal CommitPoint LatestCommitPoint; + internal readonly Functions functions; + + internal readonly AsyncFasterSession FasterSession; + internal ClientSession( - FasterKV fht, - FasterKV.FasterExecutionContext ctx, + FasterKV fht, + FasterKV.FasterExecutionContext ctx, + Functions functions, bool supportAsync) { this.fht = fht; this.ctx = ctx; - this.SupportAsync = supportAsync; + this.functions = functions; + SupportAsync = supportAsync; LatestCommitPoint = new CommitPoint { UntilSerialNo = -1, ExcludedSerialNos = null }; + FasterSession = new AsyncFasterSession(this); + // Session runs on a single thread if (!supportAsync) UnsafeResumeThread(); @@ -76,7 +85,7 @@ public Status Read(ref Key key, ref Input input, ref Output output, Context user if (SupportAsync) UnsafeResumeThread(); try { - return fht.ContextRead(ref key, ref input, ref output, userContext, serialNo, ctx); + return fht.ContextRead(ref key, ref input, ref output, userContext, FasterSession, serialNo, ctx); } finally { @@ -94,10 +103,10 @@ public Status Read(ref Key key, ref Input input, ref Output output, Context user /// /// ReadAsyncResult - call CompleteRead on the return value to complete the read operation [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTask.ReadAsyncResult> ReadAsync(ref Key key, ref Input input, Context context = default, CancellationToken token = default) + public ValueTask.ReadAsyncResult> ReadAsync(ref Key key, ref Input input, Context context = default, CancellationToken token = default) { return fht.ReadAsync(this, ref key, ref input, context, token); - } + } /// /// Upsert operation @@ -113,7 +122,7 @@ public Status Upsert(ref Key key, ref Value desiredValue, Context userContext, l if (SupportAsync) UnsafeResumeThread(); try { - return fht.ContextUpsert(ref key, ref desiredValue, userContext, serialNo, ctx); + return fht.ContextUpsert(ref key, ref desiredValue, userContext, FasterSession, serialNo, ctx); } finally { @@ -139,20 +148,15 @@ public ValueTask UpsertAsync(ref Key key, ref Value desiredValue, Context contex return default; return SlowUpsertAsync(this, waitForCommit, status, token); - - static async ValueTask SlowUpsertAsync( - ClientSession @this, - bool waitForCommit, Status status, CancellationToken token - ) - { - if (status == Status.PENDING) - await @this.CompletePendingAsync(waitForCommit, token); - else if (waitForCommit) - await @this.WaitForCommitAsync(token); - } } - + private static async ValueTask SlowUpsertAsync(ClientSession @this, bool waitForCommit, Status status, CancellationToken token) + { + if (status == Status.PENDING) + await @this.CompletePendingAsync(waitForCommit, token); + else if (waitForCommit) + await @this.WaitForCommitAsync(token); + } /// /// RMW operation @@ -168,7 +172,7 @@ public Status RMW(ref Key key, ref Input input, Context userContext, long serial if (SupportAsync) UnsafeResumeThread(); try { - return fht.ContextRMW(ref key, ref input, userContext, serialNo, ctx); + return fht.ContextRMW(ref key, ref input, userContext, FasterSession, serialNo, ctx); } finally { @@ -195,21 +199,16 @@ public ValueTask RMWAsync(ref Key key, ref Input input, Context context = defaul return default; return SlowRMWAsync(this, waitForCommit, status, token); - - static async ValueTask SlowRMWAsync( - ClientSession @this, - bool waitForCommit, Status status, CancellationToken token - ) - { - - if (status == Status.PENDING) - await @this.CompletePendingAsync(waitForCommit, token); - else if (waitForCommit) - await @this.WaitForCommitAsync(token); - } } + private static async ValueTask SlowRMWAsync(ClientSession @this, bool waitForCommit, Status status, CancellationToken token) + { + if (status == Status.PENDING) + await @this.CompletePendingAsync(waitForCommit, token); + else if (waitForCommit) + await @this.WaitForCommitAsync(token); + } /// /// Delete operation @@ -224,7 +223,7 @@ public Status Delete(ref Key key, Context userContext, long serialNo) if (SupportAsync) UnsafeResumeThread(); try { - return fht.ContextDelete(ref key, userContext, serialNo, ctx); + return fht.ContextDelete(ref key, userContext, FasterSession, serialNo, ctx); } finally { @@ -249,21 +248,16 @@ public ValueTask DeleteAsync(ref Key key, Context context = default, bool waitFo return default; return SlowDeleteAsync(this, waitForCommit, status, token); - - static async ValueTask SlowDeleteAsync( - ClientSession @this, - bool waitForCommit, Status status, CancellationToken token - ) - { - - if (status == Status.PENDING) - await @this.CompletePendingAsync(waitForCommit, token); - else if (waitForCommit) - await @this.WaitForCommitAsync(token); - } } + private static async ValueTask SlowDeleteAsync(ClientSession @this, bool waitForCommit, Status status, CancellationToken token) + { + if (status == Status.PENDING) + await @this.CompletePendingAsync(waitForCommit, token); + else if (waitForCommit) + await @this.WaitForCommitAsync(token); + } /// /// Experimental feature @@ -276,7 +270,7 @@ static async ValueTask SlowDeleteAsync( /// Status internal Status ContainsKeyInMemory(ref Key key, long fromAddress = -1) { - return fht.InternalContainsKeyInMemory(ref key, ctx, fromAddress); + return fht.InternalContainsKeyInMemory(ref key, ctx, FasterSession, fromAddress); } /// @@ -305,7 +299,7 @@ public IEnumerable GetPendingRequests() public void Refresh() { if (SupportAsync) UnsafeResumeThread(); - fht.InternalRefresh(ctx, this); + fht.InternalRefresh(ctx, FasterSession); if (SupportAsync) UnsafeSuspendThread(); } @@ -321,7 +315,7 @@ public bool CompletePending(bool spinWait = false, bool spinWaitForCommit = fals if (SupportAsync) UnsafeResumeThread(); try { - var result = fht.InternalCompletePending(ctx, spinWait); + var result = fht.InternalCompletePending(ctx, FasterSession, spinWait); if (spinWaitForCommit) { if (spinWait != true) @@ -330,10 +324,10 @@ public bool CompletePending(bool spinWait = false, bool spinWaitForCommit = fals } do { - fht.InternalCompletePending(ctx, spinWait); + fht.InternalCompletePending(ctx, FasterSession, spinWait); if (fht.InRestPhase()) { - fht.InternalCompletePending(ctx, spinWait); + fht.InternalCompletePending(ctx, FasterSession, spinWait); return true; } } while (spinWait); @@ -419,7 +413,7 @@ public async ValueTask WaitForCommitAsync(CancellationToken token = default) internal void UnsafeResumeThread() { fht.epoch.Resume(); - fht.InternalRefresh(ctx, this); + fht.InternalRefresh(ctx, FasterSession); } /// @@ -431,5 +425,131 @@ internal void UnsafeSuspendThread() fht.epoch.Suspend(); } + void IClientSession.AtomicSwitch(int version) + { + fht.AtomicSwitch(ctx, ctx.prevCtx, version); + } + + /// + /// State storage for the completion of an async Read, or the result if the read was completed synchronously + /// + public struct ReadAsyncResult + { + readonly Status status; + readonly Output output; + + readonly FasterKV.ReadAsyncInternal readAsyncInternal; + + internal ReadAsyncResult(Status status, Output output) + { + this.status = status; + this.output = output; + readAsyncInternal = default; + } + + internal ReadAsyncResult( + FasterKV fasterKV, + ClientSession clientSession, + FasterKV.PendingContext pendingContext, AsyncIOContext diskRequest) + { + status = Status.PENDING; + output = default; + readAsyncInternal = new FasterKV.ReadAsyncInternal(fasterKV, clientSession, pendingContext, diskRequest); + } + + /// + /// Complete the read operation, after any I/O is completed. + /// + /// The read result, or throws an exception if error encountered. + public (Status, Output) CompleteRead() + { + if (status != Status.PENDING) + return (status, output); + + return readAsyncInternal.CompleteRead(); + } + } + + // This is a struct to allow JIT to inline calls (and bypass default interface call mechanism) + internal struct AsyncFasterSession : IFasterSession + { + private readonly ClientSession _clientSession; + + public AsyncFasterSession(ClientSession clientSession) + { + _clientSession = clientSession; + } + + public void CheckpointCompletionCallback(string guid, CommitPoint commitPoint) + { + _clientSession.functions.CheckpointCompletionCallback(guid, commitPoint); + _clientSession.LatestCommitPoint = commitPoint; + } + + public void ConcurrentReader(ref Key key, ref Input input, ref Value value, ref Output dst) + { + _clientSession.functions.ConcurrentReader(ref key, ref input, ref value, ref dst); + } + + public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) + { + return _clientSession.functions.ConcurrentWriter(ref key, ref src, ref dst); + } + + public void CopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue) + { + _clientSession.functions.CopyUpdater(ref key, ref input, ref oldValue, ref newValue); + } + + public void DeleteCompletionCallback(ref Key key, Context ctx) + { + _clientSession.functions.DeleteCompletionCallback(ref key, ctx); + } + + public void InitialUpdater(ref Key key, ref Input input, ref Value value) + { + _clientSession.functions.InitialUpdater(ref key, ref input, ref value); + } + + public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value) + { + return _clientSession.functions.InPlaceUpdater(ref key, ref input, ref value); + } + + public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, Context ctx, Status status) + { + _clientSession.functions.ReadCompletionCallback(ref key, ref input, ref output, ctx, status); + } + + public void RMWCompletionCallback(ref Key key, ref Input input, Context ctx, Status status) + { + _clientSession.functions.RMWCompletionCallback(ref key, ref input, ctx, status); + } + + public void SingleReader(ref Key key, ref Input input, ref Value value, ref Output dst) + { + _clientSession.functions.SingleReader(ref key, ref input, ref value, ref dst); + } + + public void SingleWriter(ref Key key, ref Value src, ref Value dst) + { + _clientSession.functions.SingleWriter(ref key, ref src, ref dst); + } + + public void UnsafeResumeThread() + { + _clientSession.UnsafeResumeThread(); + } + + public void UnsafeSuspendThread() + { + _clientSession.UnsafeSuspendThread(); + } + + public void UpsertCompletionCallback(ref Key key, ref Value value, Context ctx) + { + _clientSession.functions.UpsertCompletionCallback(ref key, ref value, ctx); + } + } } } diff --git a/cs/src/core/ClientSession/FASTERAsync.cs b/cs/src/core/ClientSession/FASTERAsync.cs index d351cd05d..f2452f553 100644 --- a/cs/src/core/ClientSession/FASTERAsync.cs +++ b/cs/src/core/ClientSession/FASTERAsync.cs @@ -17,14 +17,9 @@ namespace FASTER.core /// /// Key /// Value - /// Input - /// Output - /// Context - /// Functions - public partial class FasterKV : FasterBase, IFasterKV + public partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { /// @@ -33,7 +28,8 @@ public partial class FasterKV : F /// /// /// - internal async ValueTask ReadyToCompletePendingAsync(ClientSession clientSession, CancellationToken token = default) + internal async ValueTask ReadyToCompletePendingAsync(ClientSession clientSession, CancellationToken token = default) + where Functions : IFunctions { #region Previous pending requests if (!RelaxedCPR) @@ -55,7 +51,8 @@ internal async ValueTask ReadyToCompletePendingAsync(ClientSession /// - internal async ValueTask CompletePendingAsync(ClientSession clientSession, CancellationToken token = default) + internal async ValueTask CompletePendingAsync(ClientSession clientSession, CancellationToken token = default) + where Functions : IFunctions { bool done = true; @@ -69,12 +66,12 @@ internal async ValueTask CompletePendingAsync(ClientSession 0) { - InternalCompleteRetryRequests(clientSession.ctx.prevCtx, clientSession.ctx, clientSession); + InternalCompleteRetryRequests(clientSession.ctx.prevCtx, clientSession.ctx, clientSession.FasterSession); } done &= (clientSession.ctx.prevCtx.HasNoPendingRequests); @@ -82,8 +79,8 @@ internal async ValueTask CompletePendingAsync(ClientSession + where Functions : IFunctions { const int Completed = 1; const int Pending = 0; ExceptionDispatchInfo _exception; - readonly FasterKV _fasterKV; + readonly FasterKV _fasterKV; readonly ClientSession _clientSession; - PendingContext _pendingContext; + PendingContext _pendingContext; AsyncIOContext _diskRequest; int CompletionComputeStatus; - internal ReadAsyncInternal(FasterKV fasterKV, ClientSession clientSession, PendingContext pendingContext, AsyncIOContext diskRequest) + internal ReadAsyncInternal(FasterKV fasterKV, ClientSession clientSession, PendingContext pendingContext, AsyncIOContext diskRequest) { _exception = default; _fasterKV = fasterKV; @@ -131,7 +129,7 @@ internal ReadAsyncInternal(FasterKV /// State storage for the completion of an async Read, or the result if the read was completed synchronously /// - public struct ReadAsyncResult + public struct ReadAsyncResult + where Functions : IFunctions { - readonly Status status; - readonly Output output; + internal readonly Status status; + internal readonly Output output; - readonly ReadAsyncInternal readAsyncInternal; + internal readonly ReadAsyncInternal readAsyncInternal; internal ReadAsyncResult(Status status, Output output) { @@ -173,13 +172,13 @@ internal ReadAsyncResult(Status status, Output output) } internal ReadAsyncResult( - FasterKV fasterKV, - ClientSession clientSession, - PendingContext pendingContext, AsyncIOContext diskRequest) + FasterKV fasterKV, + ClientSession clientSession, + PendingContext pendingContext, AsyncIOContext diskRequest) { status = Status.PENDING; output = default; - readAsyncInternal = new ReadAsyncInternal(fasterKV, clientSession, pendingContext, diskRequest); + readAsyncInternal = new ReadAsyncInternal(fasterKV, clientSession, pendingContext, diskRequest); } /// @@ -197,10 +196,11 @@ internal ReadAsyncResult( [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal ValueTask ReadAsync(ClientSession clientSession, + internal ValueTask> ReadAsync(ClientSession clientSession, ref Key key, ref Input input, Context context = default, CancellationToken token = default) + where Functions : IFunctions { - var pcontext = default(PendingContext); + var pcontext = default(PendingContext); Output output = default; OperationStatus internalStatus; var nextSerialNum = clientSession.ctx.serialNum + 1; @@ -210,15 +210,15 @@ internal ValueTask ReadAsync(ClientSession(new ReadAsyncResult((Status)internalStatus, output)); + return new ValueTask>(new ReadAsyncResult((Status)internalStatus, output)); } if (internalStatus == OperationStatus.CPR_SHIFT_DETECTED) { - SynchronizeEpoch(clientSession.ctx, clientSession.ctx, ref pcontext); + SynchronizeEpoch(clientSession.ctx, clientSession.ctx, ref pcontext, clientSession.FasterSession); goto TryReadAgain; } } @@ -229,42 +229,43 @@ internal ValueTask ReadAsync(ClientSession SlowReadAsync( - FasterKV @this, - ClientSession clientSession, - PendingContext pendingContext, CancellationToken token = default) - { - var diskRequest = @this.ScheduleGetFromDisk(clientSession.ctx, ref pendingContext); - clientSession.ctx.ioPendingRequests.Add(pendingContext.id, pendingContext); - clientSession.ctx.asyncPendingCount++; - clientSession.ctx.pendingReads.Add(); - - try - { - token.ThrowIfCancellationRequested(); + private static async ValueTask> SlowReadAsync( + FasterKV @this, + ClientSession clientSession, + PendingContext pendingContext, CancellationToken token = default) + where Functions : IFunctions + { + var diskRequest = @this.ScheduleGetFromDisk(clientSession.ctx, ref pendingContext); + clientSession.ctx.ioPendingRequests.Add(pendingContext.id, pendingContext); + clientSession.ctx.asyncPendingCount++; + clientSession.ctx.pendingReads.Add(); - if (@this.epoch.ThisInstanceProtected()) - throw new NotSupportedException("Async operations not supported over protected epoch"); + try + { + token.ThrowIfCancellationRequested(); - diskRequest = await diskRequest.asyncOperation.ValueTaskOfT; - } - catch - { - clientSession.ctx.ioPendingRequests.Remove(pendingContext.id); - clientSession.ctx.asyncPendingCount--; - throw; - } - finally - { - clientSession.ctx.pendingReads.Remove(); - } + if (@this.epoch.ThisInstanceProtected()) + throw new NotSupportedException("Async operations not supported over protected epoch"); - return new ReadAsyncResult(@this, clientSession, pendingContext, diskRequest); + diskRequest = await diskRequest.asyncOperation.ValueTaskOfT; + } + catch + { + clientSession.ctx.ioPendingRequests.Remove(pendingContext.id); + clientSession.ctx.asyncPendingCount--; + throw; } + finally + { + clientSession.ctx.pendingReads.Remove(); + } + + return new ReadAsyncResult(@this, clientSession, pendingContext, diskRequest); } - internal bool AtomicSwitch(FasterExecutionContext fromCtx, FasterExecutionContext toCtx, int version) + internal bool AtomicSwitch(FasterExecutionContext fromCtx, FasterExecutionContext toCtx, int version) { lock (toCtx) { diff --git a/cs/src/core/ClientSession/FASTERClientSession.cs b/cs/src/core/ClientSession/FASTERClientSession.cs index 17e8b4b80..2e38ffb04 100644 --- a/cs/src/core/ClientSession/FASTERClientSession.cs +++ b/cs/src/core/ClientSession/FASTERClientSession.cs @@ -9,64 +9,157 @@ namespace FASTER.core { - public unsafe partial class FasterKV : FasterBase, IFasterKV + public unsafe partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { - internal Dictionary> _activeSessions; + internal Dictionary _activeSessions; + + /// + /// Client session type helper + /// + /// + /// + /// + public struct ClientSessionBuilder + { + private readonly FasterKV _fasterKV; + + internal ClientSessionBuilder(FasterKV fasterKV) + { + _fasterKV = fasterKV; + } + + /// + /// Start a new client session with FASTER. + /// + /// Callback functions + /// ID/name of session (auto-generated if not provided) + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + public ClientSession NewSession(Functions functions, string sessionId = null, bool threadAffinitized = false) + where Functions : IFunctions + { + return _fasterKV.NewSession(functions, sessionId, threadAffinitized); + } + + /// + /// Resume (continue) prior client session with FASTER, used during + /// recovery from failure. + /// + /// Callback functions + /// ID/name of previous session to resume + /// Prior commit point of durability for session + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + public ClientSession ResumeSession(Functions functions, string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + where Functions : IFunctions + { + return _fasterKV.ResumeSession(functions, sessionId, out commitPoint, threadAffinitized); + } + } + + /// + /// Helper method to specify Input, Output and Context for callback functions + /// + /// + /// + /// + /// + public ClientSessionBuilder For() + { + return new ClientSessionBuilder(this); + } + + /// + /// Start a new client session with FASTER. + /// For performance reasons this overload is not recommended if functions is value type (struct). + /// + /// Callback functions + /// ID/name of session (auto-generated if not provided) + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + public ClientSession> NewSession(IFunctions functions, string sessionId = null, bool threadAffinitized = false) + { + return NewSession>(functions, sessionId, threadAffinitized); + } /// /// Start a new client session with FASTER. /// + /// Callback functions /// ID/name of session (auto-generated if not provided) /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. /// Session instance - public ClientSession NewSession(string sessionId = null, bool threadAffinitized = false) + public ClientSession NewSession(Functions functions, string sessionId = null, bool threadAffinitized = false) + where Functions : IFunctions { + if (functions == null) + throw new ArgumentNullException(nameof(functions)); + if (!threadAffinitized) UseRelaxedCPR(); if (sessionId == null) sessionId = Guid.NewGuid().ToString(); - var ctx = new FasterExecutionContext(); + var ctx = new FasterExecutionContext(); InitContext(ctx, sessionId); - var prevCtx = new FasterExecutionContext(); + var prevCtx = new FasterExecutionContext(); InitContext(prevCtx, sessionId); prevCtx.version--; ctx.prevCtx = prevCtx; if (_activeSessions == null) - Interlocked.CompareExchange(ref _activeSessions, new Dictionary>(), null); + Interlocked.CompareExchange(ref _activeSessions, new Dictionary(), null); - var session = new ClientSession(this, ctx, !threadAffinitized); + var session = new ClientSession(this, ctx, functions, !threadAffinitized); lock (_activeSessions) _activeSessions.Add(sessionId, session); return session; } + /// + /// Resume (continue) prior client session with FASTER, used during + /// recovery from failure. + /// For performance reasons this overload is not recommended if functions is value type (struct). + /// + /// Callback functions + /// ID/name of previous session to resume + /// Prior commit point of durability for session + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + public ClientSession> ResumeSession(IFunctions functions, string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + { + return ResumeSession>(functions, sessionId, out commitPoint, threadAffinitized); + } + /// /// Resume (continue) prior client session with FASTER, used during /// recovery from failure. /// + /// Callback functions /// ID/name of previous session to resume /// Prior commit point of durability for session /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. /// Session instance - public ClientSession ResumeSession(string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + public ClientSession ResumeSession(Functions functions, string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + where Functions : IFunctions { + if (functions == null) + throw new ArgumentNullException(nameof(functions)); + if (!threadAffinitized) UseRelaxedCPR(); - commitPoint = InternalContinue(sessionId, out FasterExecutionContext ctx); + commitPoint = InternalContinue(sessionId, out var ctx); if (commitPoint.UntilSerialNo == -1) throw new Exception($"Unable to find session {sessionId} to recover"); - var session = new ClientSession(this, ctx, !threadAffinitized); + var session = new ClientSession(this, ctx, functions, !threadAffinitized); if (_activeSessions == null) - Interlocked.CompareExchange(ref _activeSessions, new Dictionary>(), null); + Interlocked.CompareExchange(ref _activeSessions, new Dictionary(), null); lock (_activeSessions) _activeSessions.Add(sessionId, session); return session; diff --git a/cs/src/core/ClientSession/IClientSession.cs b/cs/src/core/ClientSession/IClientSession.cs new file mode 100644 index 000000000..99cf8ebbc --- /dev/null +++ b/cs/src/core/ClientSession/IClientSession.cs @@ -0,0 +1,13 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT license. + +#pragma warning disable 0162 + + +namespace FASTER.core +{ + internal interface IClientSession + { + void AtomicSwitch(int version); + } +} diff --git a/cs/src/core/Index/Common/AddressInfo.cs b/cs/src/core/Index/Common/AddressInfo.cs index 19dc879c4..a33045ded 100644 --- a/cs/src/core/Index/Common/AddressInfo.cs +++ b/cs/src/core/Index/Common/AddressInfo.cs @@ -42,7 +42,7 @@ public static string ToString(AddressInfo* info) public long Size { - get + readonly get { int multiplier = (int)((((long)word & kMultiplierMaskInWord) >> (kAddressBits + kSizeBits)) & kMultiplierMaskInInteger); return (multiplier == 0 ? 512 : 1<<20)*((((long)word & kSizeMaskInWord) >> kAddressBits) & kSizeMaskInInteger); diff --git a/cs/src/core/Index/Common/Contexts.cs b/cs/src/core/Index/Common/Contexts.cs index 03630ad72..f2103a0a6 100644 --- a/cs/src/core/Index/Common/Contexts.cs +++ b/cs/src/core/Index/Common/Contexts.cs @@ -6,6 +6,7 @@ using System.Collections.Generic; using System.Diagnostics; using System.IO; +using System.Linq; using System.Runtime.CompilerServices; using System.Threading; @@ -62,12 +63,11 @@ public void Load(StreamReader reader) } } - public partial class FasterKV : FasterBase, IFasterKV + public partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { - internal struct PendingContext + internal struct PendingContext { // User provided information internal OperationType type; @@ -93,13 +93,13 @@ public void Dispose() } } - internal sealed class FasterExecutionContext : SerializedFasterExecutionContext + internal sealed class FasterExecutionContext : SerializedFasterExecutionContext { public Phase phase; public bool[] markers; public long totalPending; - public Queue retryRequests; - public Dictionary ioPendingRequests; + public Queue> retryRequests; + public Dictionary> ioPendingRequests; public AsyncCountDown pendingReads; public AsyncQueue> readyResponses; public List excludedSerialNos; @@ -116,7 +116,7 @@ public bool HasNoPendingRequests } } - public FasterExecutionContext prevCtx; + public FasterExecutionContext prevCtx; } } @@ -257,10 +257,10 @@ public void Initialize(StreamReader reader) exclusions.Add(long.Parse(reader.ReadLine())); continueTokens.TryAdd(guid, new CommitPoint - { - UntilSerialNo = serialno, - ExcludedSerialNos = exclusions - }); + { + UntilSerialNo = serialno, + ExcludedSerialNos = exclusions + }); } // Read object log segment offsets @@ -289,8 +289,8 @@ internal void Recover(Guid token, ICheckpointManager checkpointManager) if (metadata == null) throw new FasterException("Invalid log commit metadata for ID " + token.ToString()); - using StreamReader s = new StreamReader(new MemoryStream(metadata)); - Initialize(s); + using (StreamReader s = new StreamReader(new MemoryStream(metadata))) + Initialize(s); } /// @@ -306,39 +306,41 @@ public void Reset() /// public byte[] ToByteArray() { - using MemoryStream ms = new MemoryStream(); - using (StreamWriter writer = new StreamWriter(ms)) + using (MemoryStream ms = new MemoryStream()) { - writer.WriteLine(guid); - writer.WriteLine(useSnapshotFile); - writer.WriteLine(version); - writer.WriteLine(flushedLogicalAddress); - writer.WriteLine(startLogicalAddress); - writer.WriteLine(finalLogicalAddress); - writer.WriteLine(headAddress); - writer.WriteLine(beginAddress); - - writer.WriteLine(checkpointTokens.Count); - foreach (var kvp in checkpointTokens) + using (StreamWriter writer = new StreamWriter(ms)) { - writer.WriteLine(kvp.Key); - writer.WriteLine(kvp.Value.UntilSerialNo); - writer.WriteLine(kvp.Value.ExcludedSerialNos.Count); - foreach (long item in kvp.Value.ExcludedSerialNos) - writer.WriteLine(item); - } + writer.WriteLine(guid); + writer.WriteLine(useSnapshotFile); + writer.WriteLine(version); + writer.WriteLine(flushedLogicalAddress); + writer.WriteLine(startLogicalAddress); + writer.WriteLine(finalLogicalAddress); + writer.WriteLine(headAddress); + writer.WriteLine(beginAddress); + + writer.WriteLine(checkpointTokens.Count); + foreach (var kvp in checkpointTokens) + { + writer.WriteLine(kvp.Key); + writer.WriteLine(kvp.Value.UntilSerialNo); + writer.WriteLine(kvp.Value.ExcludedSerialNos.Count); + foreach (long item in kvp.Value.ExcludedSerialNos) + writer.WriteLine(item); + } - // Write object log segment offsets - writer.WriteLine(objectLogSegmentOffsets == null ? 0 : objectLogSegmentOffsets.Length); - if (objectLogSegmentOffsets != null) - { - for (int i = 0; i < objectLogSegmentOffsets.Length; i++) + // Write object log segment offsets + writer.WriteLine(objectLogSegmentOffsets == null ? 0 : objectLogSegmentOffsets.Length); + if (objectLogSegmentOffsets != null) { - writer.WriteLine(objectLogSegmentOffsets[i]); + for (int i = 0; i < objectLogSegmentOffsets.Length; i++) + { + writer.WriteLine(objectLogSegmentOffsets[i]); + } } } + return ms.ToArray(); } - return ms.ToArray(); } /// @@ -356,10 +358,13 @@ public readonly void DebugPrint() Debug.WriteLine("Begin Address: {0}", beginAddress); Debug.WriteLine("Num sessions recovered: {0}", continueTokens.Count); Debug.WriteLine("Recovered sessions: "); - foreach (var sessionInfo in continueTokens) + foreach (var sessionInfo in continueTokens.Take(10)) { Debug.WriteLine("{0}: {1}", sessionInfo.Key, sessionInfo.Value); } + + if (continueTokens.Count > 10) + Debug.WriteLine("... {0} skipped", continueTokens.Count - 10); } } @@ -444,25 +449,27 @@ public void Recover(Guid guid, ICheckpointManager checkpointManager) var metadata = checkpointManager.GetIndexCommitMetadata(guid); if (metadata == null) throw new FasterException("Invalid index commit metadata for ID " + guid.ToString()); - using StreamReader s = new StreamReader(new MemoryStream(metadata)); - Initialize(s); + using (StreamReader s = new StreamReader(new MemoryStream(metadata))) + Initialize(s); } public readonly byte[] ToByteArray() { - using MemoryStream ms = new MemoryStream(); - using (var writer = new StreamWriter(ms)) + using (MemoryStream ms = new MemoryStream()) { + using (var writer = new StreamWriter(ms)) + { - writer.WriteLine(token); - writer.WriteLine(table_size); - writer.WriteLine(num_ht_bytes); - writer.WriteLine(num_ofb_bytes); - writer.WriteLine(num_buckets); - writer.WriteLine(startLogicalAddress); - writer.WriteLine(finalLogicalAddress); + writer.WriteLine(token); + writer.WriteLine(table_size); + writer.WriteLine(num_ht_bytes); + writer.WriteLine(num_ofb_bytes); + writer.WriteLine(num_buckets); + writer.WriteLine(startLogicalAddress); + writer.WriteLine(finalLogicalAddress); + } + return ms.ToArray(); } - return ms.ToArray(); } public readonly void DebugPrint() diff --git a/cs/src/core/Index/Common/LogSettings.cs b/cs/src/core/Index/Common/LogSettings.cs index 91f6c391f..060afc2ce 100644 --- a/cs/src/core/Index/Common/LogSettings.cs +++ b/cs/src/core/Index/Common/LogSettings.cs @@ -37,6 +37,14 @@ public interface IVariableLengthStruct /// int GetLength(ref T t); + /// + /// Actual length of object + /// + /// + /// + /// + int GetLength(ref T t, ref Input input); + /// /// Average length of objects, make sure this includes the object /// header needed to compute the actual object length @@ -53,7 +61,6 @@ public interface IVariableLengthStruct int GetInitialLength(ref Input input); } - /// /// Length specification for fixed size (normal) structs /// @@ -82,6 +89,14 @@ public interface IVariableLengthStruct /// /// public int GetLength(ref T t) => size; + + /// + /// Actual length of object + /// + /// + /// + /// + public int GetLength(ref T t, ref Input input) => size; } /// diff --git a/cs/src/core/Index/FASTER/FASTER.cs b/cs/src/core/Index/FASTER/FASTER.cs index 4f7990e02..271c20b7d 100644 --- a/cs/src/core/Index/FASTER/FASTER.cs +++ b/cs/src/core/Index/FASTER/FASTER.cs @@ -10,13 +10,11 @@ namespace FASTER.core { - public partial class FasterKV : FasterBase, - IFasterKV + public partial class FasterKV : FasterBase, + IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { - internal readonly Functions functions; internal readonly AllocatorBase hlog; private readonly AllocatorBase readcache; private readonly IFasterEqualityComparer comparer; @@ -54,13 +52,13 @@ public partial class FasterKV : F /// /// Hybrid log used by this FASTER instance /// - public LogAccessor Log { get; } + public LogAccessor Log { get; } /// /// Read cache used by this FASTER instance /// - public LogAccessor ReadCache { get; } - + public LogAccessor ReadCache { get; } + internal ConcurrentDictionary _recoveredSessions; /// @@ -69,11 +67,10 @@ public partial class FasterKV : F /// Size of core index (#cache lines) /// FASTER equality comparer for key /// - /// Callback functions /// Log settings /// Checkpoint settings /// Serializer settings - public FasterKV(long size, Functions functions, LogSettings logSettings, + public FasterKV(long size, LogSettings logSettings, CheckpointSettings checkpointSettings = null, SerializerSettings serializerSettings = null, IFasterEqualityComparer comparer = null, VariableLengthStructSettings variableLengthStructSettings = null) @@ -106,7 +103,6 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, FoldOverSnapshot = checkpointSettings.CheckPointType == core.CheckpointType.FoldOver; CopyReadsToTail = logSettings.CopyReadsToTail; - this.functions = functions; if (logSettings.ReadCacheSettings != null) { @@ -120,7 +116,7 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, { hlog = new VariableLengthBlittableAllocator(logSettings, variableLengthStructSettings, this.comparer, null, epoch); - Log = new LogAccessor(this, hlog); + Log = new LogAccessor(this, hlog); if (UseReadCache) { readcache = new VariableLengthBlittableAllocator( @@ -132,13 +128,13 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, MutableFraction = 1 - logSettings.ReadCacheSettings.SecondChanceFraction }, variableLengthStructSettings, this.comparer, ReadCacheEvict, epoch); readcache.Initialize(); - ReadCache = new LogAccessor(this, readcache); + ReadCache = new LogAccessor(this, readcache); } } else { hlog = new BlittableAllocator(logSettings, this.comparer, null, epoch); - Log = new LogAccessor(this, hlog); + Log = new LogAccessor(this, hlog); if (UseReadCache) { readcache = new BlittableAllocator( @@ -150,7 +146,7 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, MutableFraction = 1 - logSettings.ReadCacheSettings.SecondChanceFraction }, this.comparer, ReadCacheEvict, epoch); readcache.Initialize(); - ReadCache = new LogAccessor(this, readcache); + ReadCache = new LogAccessor(this, readcache); } } } @@ -159,7 +155,7 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, WriteDefaultOnDelete = true; hlog = new GenericAllocator(logSettings, serializerSettings, this.comparer, null, epoch); - Log = new LogAccessor(this, hlog); + Log = new LogAccessor(this, hlog); if (UseReadCache) { readcache = new GenericAllocator( @@ -171,18 +167,18 @@ public FasterKV(long size, Functions functions, LogSettings logSettings, MutableFraction = 1 - logSettings.ReadCacheSettings.SecondChanceFraction }, serializerSettings, this.comparer, ReadCacheEvict, epoch); readcache.Initialize(); - ReadCache = new LogAccessor(this, readcache); + ReadCache = new LogAccessor(this, readcache); } } hlog.Initialize(); - sectorSize = (int) logSettings.LogDevice.SectorSize; + sectorSize = (int)logSettings.LogDevice.SectorSize; Initialize(size, sectorSize); - _systemState = default; - _systemState.phase = Phase.REST; - _systemState.version = 1; + systemState = default; + systemState.phase = Phase.REST; + systemState.version = 1; } /// @@ -279,52 +275,54 @@ public async ValueTask CompleteCheckpointAsync(CancellationToken token = default while (true) { - var systemState = _systemState; + var systemState = this.systemState; if (systemState.phase == Phase.REST || systemState.phase == Phase.PREPARE_GROW || systemState.phase == Phase.IN_PROGRESS_GROW) return; - await ThreadStateMachineStep(null, null, true, token); + // TODO: Do we need access to functions here? + // If yes then move this to either faster legacy or client session. + await ThreadStateMachineStep(null, NullFasterSession.Instance, true, token); } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextRead(ref Key key, ref Input input, ref Output output, Context context, long serialNo, - FasterExecutionContext sessionCtx) + internal Status ContextRead(ref Key key, ref Input input, ref Output output, Context context, FasterSession fasterSession, long serialNo, + FasterExecutionContext sessionCtx) + where FasterSession : IFasterSession { - var pcontext = default(PendingContext); - var internalStatus = InternalRead(ref key, ref input, ref output, ref context, ref pcontext, sessionCtx, - serialNo); + var pcontext = default(PendingContext); + var internalStatus = InternalRead(ref key, ref input, ref output, ref context, ref pcontext, fasterSession, sessionCtx, serialNo); Status status; if (internalStatus == OperationStatus.SUCCESS || internalStatus == OperationStatus.NOTFOUND) { - status = (Status) internalStatus; + status = (Status)internalStatus; } else { - status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, internalStatus); + status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, fasterSession, internalStatus); } sessionCtx.serialNum = serialNo; return status; } - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextUpsert(ref Key key, ref Value value, Context context, long serialNo, - FasterExecutionContext sessionCtx) + internal Status ContextUpsert(ref Key key, ref Value value, Context context, FasterSession fasterSession, long serialNo, + FasterExecutionContext sessionCtx) + where FasterSession : IFasterSession { - var pcontext = default(PendingContext); - var internalStatus = InternalUpsert(ref key, ref value, ref context, ref pcontext, sessionCtx, serialNo); + var pcontext = default(PendingContext); + var internalStatus = InternalUpsert(ref key, ref value, ref context, ref pcontext, fasterSession, sessionCtx, serialNo); Status status; if (internalStatus == OperationStatus.SUCCESS || internalStatus == OperationStatus.NOTFOUND) { - status = (Status) internalStatus; + status = (Status)internalStatus; } else { - status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, internalStatus); + status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, fasterSession, internalStatus); } sessionCtx.serialNum = serialNo; @@ -332,19 +330,20 @@ internal Status ContextUpsert(ref Key key, ref Value value, Context context, lon } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextRMW(ref Key key, ref Input input, Context context, long serialNo, - FasterExecutionContext sessionCtx) + internal Status ContextRMW(ref Key key, ref Input input, Context context, FasterSession fasterSession, long serialNo, + FasterExecutionContext sessionCtx) + where FasterSession : IFasterSession { - var pcontext = default(PendingContext); - var internalStatus = InternalRMW(ref key, ref input, ref context, ref pcontext, sessionCtx, serialNo); + var pcontext = default(PendingContext); + var internalStatus = InternalRMW(ref key, ref input, ref context, ref pcontext, fasterSession, sessionCtx, serialNo); Status status; if (internalStatus == OperationStatus.SUCCESS || internalStatus == OperationStatus.NOTFOUND) { - status = (Status) internalStatus; + status = (Status)internalStatus; } else { - status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, internalStatus); + status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, fasterSession, internalStatus); } sessionCtx.serialNum = serialNo; @@ -352,25 +351,30 @@ internal Status ContextRMW(ref Key key, ref Input input, Context context, long s } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status ContextDelete(ref Key key, Context context, long serialNo, FasterExecutionContext sessionCtx) + internal Status ContextDelete( + ref Key key, + Context context, + FasterSession fasterSession, + long serialNo, + FasterExecutionContext sessionCtx) + where FasterSession : IFasterSession { - var pcontext = default(PendingContext); - var internalStatus = InternalDelete(ref key, ref context, ref pcontext, sessionCtx, serialNo); + var pcontext = default(PendingContext); + var internalStatus = InternalDelete(ref key, ref context, ref pcontext, fasterSession, sessionCtx, serialNo); Status status; if (internalStatus == OperationStatus.SUCCESS || internalStatus == OperationStatus.NOTFOUND) { - status = (Status) internalStatus; + status = (Status)internalStatus; } else { - status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, internalStatus); + status = HandleOperationStatus(sessionCtx, sessionCtx, pcontext, fasterSession, internalStatus); } sessionCtx.serialNum = serialNo; return status; } - /// /// Grow the hash index /// @@ -385,8 +389,7 @@ public bool GrowIndex() /// public void Dispose() { - base.Free(); - LegacyDispose(); + Free(); hlog.Dispose(); readcache?.Dispose(); } diff --git a/cs/src/core/Index/FASTER/FASTERImpl.cs b/cs/src/core/Index/FASTER/FASTERImpl.cs index 851723872..703eae645 100644 --- a/cs/src/core/Index/FASTER/FASTERImpl.cs +++ b/cs/src/core/Index/FASTER/FASTERImpl.cs @@ -10,10 +10,9 @@ namespace FASTER.core { - public unsafe partial class FasterKV : FasterBase, IFasterKV + public unsafe partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { internal enum LatchOperation : byte { @@ -34,6 +33,7 @@ internal enum LatchOperation : byte /// Location to store output computed from input and value. /// User context for the operation, in case it goes pending. /// Pending context used internally to store the context of the operation. + /// Callback functions. /// Session context /// Operation serial number /// @@ -57,12 +57,16 @@ internal enum LatchOperation : byte /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalRead( + internal OperationStatus InternalRead( ref Key key, ref Input input, ref Output output, ref Context userContext, - ref PendingContext pendingContext, FasterExecutionContext sessionCtx, long lsn) + ref PendingContext pendingContext, + Functions fasterSession, + FasterExecutionContext sessionCtx, + long lsn) + where Functions : IFasterSession { var bucket = default(HashBucket*); var slot = default(int); @@ -74,7 +78,7 @@ internal OperationStatus InternalRead( var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); if (sessionCtx.phase != Phase.REST) - HeavyEnter(hash, sessionCtx); + HeavyEnter(hash, sessionCtx, fasterSession); #region Trace back for record in in-memory HybridLog HashBucketEntry entry = default; @@ -92,7 +96,7 @@ internal OperationStatus InternalRead( status = OperationStatus.CPR_SHIFT_DETECTED; goto CreatePendingContext; // Pivot thread } - functions.SingleReader(ref key, ref input, ref readcache.GetValue(physicalAddress), ref output); + fasterSession.SingleReader(ref key, ref input, ref readcache.GetValue(physicalAddress), ref output); return OperationStatus.SUCCESS; } @@ -148,7 +152,7 @@ internal OperationStatus InternalRead( if (hlog.GetInfo(physicalAddress).Tombstone) return OperationStatus.NOTFOUND; - functions.ConcurrentReader(ref key, ref input, ref hlog.GetValue(physicalAddress), ref output); + fasterSession.ConcurrentReader(ref key, ref input, ref hlog.GetValue(physicalAddress), ref output); return OperationStatus.SUCCESS; } @@ -158,7 +162,7 @@ internal OperationStatus InternalRead( if (hlog.GetInfo(physicalAddress).Tombstone) return OperationStatus.NOTFOUND; - functions.SingleReader(ref key, ref input, ref hlog.GetValue(physicalAddress), ref output); + fasterSession.SingleReader(ref key, ref input, ref hlog.GetValue(physicalAddress), ref output); return OperationStatus.SUCCESS; } @@ -196,10 +200,10 @@ internal OperationStatus InternalRead( return OperationStatus.NOTFOUND; } - #endregion + #endregion - #region Create pending context - CreatePendingContext: + #region Create pending context + CreatePendingContext: { pendingContext.type = OperationType.READ; pendingContext.key = hlog.GetKeyContainer(ref key); @@ -228,6 +232,7 @@ internal OperationStatus InternalRead( /// value to be updated to (or inserted if key does not exist). /// User context for the operation, in case it goes pending. /// Pending context used internally to store the context of the operation. + /// Callback functions. /// Session context /// Operation serial number /// @@ -251,10 +256,14 @@ internal OperationStatus InternalRead( /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalUpsert( + internal OperationStatus InternalUpsert( ref Key key, ref Value value, ref Context userContext, - ref PendingContext pendingContext, FasterExecutionContext sessionCtx, long lsn) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext sessionCtx, + long lsn) + where FasterSession : IFasterSession { var status = default(OperationStatus); var bucket = default(HashBucket*); @@ -268,7 +277,7 @@ internal OperationStatus InternalUpsert( var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); if (sessionCtx.phase != Phase.REST) - HeavyEnter(hash, sessionCtx); + HeavyEnter(hash, sessionCtx, fasterSession); #region Trace back for record in in-memory HybridLog var entry = default(HashBucketEntry); @@ -299,7 +308,7 @@ internal OperationStatus InternalUpsert( // Optimization for most common case if (sessionCtx.phase == Phase.REST && logicalAddress >= hlog.ReadOnlyAddress && !hlog.GetInfo(physicalAddress).Tombstone) { - if (functions.ConcurrentWriter(ref key, ref value, ref hlog.GetValue(physicalAddress))) + if (fasterSession.ConcurrentWriter(ref key, ref value, ref hlog.GetValue(physicalAddress))) { return OperationStatus.SUCCESS; } @@ -384,29 +393,29 @@ internal OperationStatus InternalUpsert( // Mutable Region: Update the record in-place if (logicalAddress >= hlog.ReadOnlyAddress && !hlog.GetInfo(physicalAddress).Tombstone) { - if (functions.ConcurrentWriter(ref key, ref value, ref hlog.GetValue(physicalAddress))) + if (fasterSession.ConcurrentWriter(ref key, ref value, ref hlog.GetValue(physicalAddress))) { status = OperationStatus.SUCCESS; goto LatchRelease; // Release shared latch (if acquired) } } - // All other regions: Create a record in the mutable region - #endregion + // All other regions: Create a record in the mutable region + #endregion - #region Create new record in the mutable region - CreateNewRecord: + #region Create new record in the mutable region + CreateNewRecord: { // Immutable region or new record var recordSize = hlog.GetRecordSize(ref key, ref value); - BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx); + BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx, fasterSession); var newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref hlog.GetInfo(newPhysicalAddress), sessionCtx.version, true, false, false, latestLogicalAddress); hlog.ShallowCopy(ref key, ref hlog.GetKey(newPhysicalAddress)); - functions.SingleWriter(ref key, ref value, + fasterSession.SingleWriter(ref key, ref value, ref hlog.GetValue(newPhysicalAddress)); var updatedEntry = default(HashBucketEntry); @@ -432,10 +441,10 @@ internal OperationStatus InternalUpsert( goto LatchRelease; } } - #endregion + #endregion - #region Create pending context - CreatePendingContext: + #region Create pending context + CreatePendingContext: { pendingContext.type = OperationType.UPSERT; pendingContext.key = hlog.GetKeyContainer(ref key); @@ -446,10 +455,10 @@ internal OperationStatus InternalUpsert( pendingContext.version = sessionCtx.version; pendingContext.serialNum = lsn; } - #endregion + #endregion - #region Latch release - LatchRelease: + #region Latch release + LatchRelease: { switch (latchOperation) { @@ -467,7 +476,7 @@ internal OperationStatus InternalUpsert( if (status == OperationStatus.RETRY_NOW) { - return InternalUpsert(ref key, ref value, ref userContext, ref pendingContext, sessionCtx, lsn); + return InternalUpsert(ref key, ref value, ref userContext, ref pendingContext, fasterSession, sessionCtx, lsn); } else { @@ -488,6 +497,7 @@ internal OperationStatus InternalUpsert( /// input used to update the value. /// user context corresponding to operation used during completion callback. /// pending context created when the operation goes pending. + /// Callback functions. /// Session context /// Operation serial number /// @@ -515,10 +525,14 @@ internal OperationStatus InternalUpsert( /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalRMW( + internal OperationStatus InternalRMW( ref Key key, ref Input input, ref Context userContext, - ref PendingContext pendingContext, FasterExecutionContext sessionCtx, long lsn) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext sessionCtx, + long lsn) + where FasterSession : IFasterSession { var recordSize = default(int); var bucket = default(HashBucket*); @@ -534,7 +548,7 @@ internal OperationStatus InternalRMW( var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); if (sessionCtx.phase != Phase.REST) - HeavyEnter(hash, sessionCtx); + HeavyEnter(hash, sessionCtx, fasterSession); #region Trace back for record in in-memory HybridLog var entry = default(HashBucketEntry); @@ -565,7 +579,7 @@ internal OperationStatus InternalRMW( // Optimization for the most common case if (sessionCtx.phase == Phase.REST && logicalAddress >= hlog.ReadOnlyAddress && !hlog.GetInfo(physicalAddress).Tombstone) { - if (functions.InPlaceUpdater(ref key, ref input, ref hlog.GetValue(physicalAddress))) + if (fasterSession.InPlaceUpdater(ref key, ref input, ref hlog.GetValue(physicalAddress))) { return OperationStatus.SUCCESS; } @@ -657,7 +671,7 @@ internal OperationStatus InternalRMW( Debug.Assert(hlog.GetInfo(physicalAddress).Version == sessionCtx.version); } - if (functions.InPlaceUpdater(ref key, ref input, ref hlog.GetValue(physicalAddress))) + if (fasterSession.InPlaceUpdater(ref key, ref input, ref hlog.GetValue(physicalAddress))) { status = OperationStatus.SUCCESS; goto LatchRelease; // Release shared latch (if acquired) @@ -710,15 +724,15 @@ internal OperationStatus InternalRMW( goto CreateNewRecord; } - #endregion + #endregion #region Create new record - CreateNewRecord: + CreateNewRecord: { recordSize = (logicalAddress < hlog.BeginAddress) ? hlog.GetInitialRecordSize(ref key, ref input) : - hlog.GetRecordSize(physicalAddress); - BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx); + hlog.GetRecordSize(physicalAddress, ref input); + BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx, fasterSession); var newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref hlog.GetInfo(newPhysicalAddress), sessionCtx.version, true, false, false, @@ -726,19 +740,19 @@ internal OperationStatus InternalRMW( hlog.ShallowCopy(ref key, ref hlog.GetKey(newPhysicalAddress)); if (logicalAddress < hlog.BeginAddress) { - functions.InitialUpdater(ref key, ref input, ref hlog.GetValue(newPhysicalAddress)); + fasterSession.InitialUpdater(ref key, ref input, ref hlog.GetValue(newPhysicalAddress)); status = OperationStatus.NOTFOUND; } else if (logicalAddress >= hlog.HeadAddress) { if (hlog.GetInfo(physicalAddress).Tombstone) { - functions.InitialUpdater(ref key, ref input, ref hlog.GetValue(newPhysicalAddress)); + fasterSession.InitialUpdater(ref key, ref input, ref hlog.GetValue(newPhysicalAddress)); status = OperationStatus.NOTFOUND; } else { - functions.CopyUpdater(ref key, ref input, + fasterSession.CopyUpdater(ref key, ref input, ref hlog.GetValue(physicalAddress), ref hlog.GetValue(newPhysicalAddress)); status = OperationStatus.SUCCESS; @@ -778,7 +792,7 @@ ref hlog.GetValue(physicalAddress), #endregion #region Create failure context - CreateFailureContext: + CreateFailureContext: { pendingContext.type = OperationType.RMW; pendingContext.key = hlog.GetKeyContainer(ref key); @@ -793,7 +807,7 @@ ref hlog.GetValue(physicalAddress), #endregion #region Latch release - LatchRelease: + LatchRelease: { switch (latchOperation) { @@ -811,7 +825,7 @@ ref hlog.GetValue(physicalAddress), if (status == OperationStatus.RETRY_NOW) { - return InternalRMW(ref key, ref input, ref userContext, ref pendingContext, sessionCtx, lsn); + return InternalRMW(ref key, ref input, ref userContext, ref pendingContext, fasterSession, sessionCtx, lsn); } else { @@ -830,6 +844,7 @@ ref hlog.GetValue(physicalAddress), /// Key of the record to be deleted. /// User context for the operation, in case it goes pending. /// Pending context used internally to store the context of the operation. + /// Callback functions. /// Session context /// Operation serial number /// @@ -853,10 +868,14 @@ ref hlog.GetValue(physicalAddress), /// /// [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal OperationStatus InternalDelete( + internal OperationStatus InternalDelete( ref Key key, ref Context userContext, - ref PendingContext pendingContext, FasterExecutionContext sessionCtx, long lsn) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext sessionCtx, + long lsn) + where FasterSession : IFasterSession { var status = default(OperationStatus); var bucket = default(HashBucket*); @@ -870,7 +889,7 @@ internal OperationStatus InternalDelete( var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); if (sessionCtx.phase != Phase.REST) - HeavyEnter(hash, sessionCtx); + HeavyEnter(hash, sessionCtx, fasterSession); #region Trace back for record in in-memory HybridLog var entry = default(HashBucketEntry); @@ -1002,7 +1021,7 @@ internal OperationStatus InternalDelete( // Write default value // Ignore return value, the record is already marked Value v = default; - functions.ConcurrentWriter(ref hlog.GetKey(physicalAddress), ref v, ref hlog.GetValue(physicalAddress)); + fasterSession.ConcurrentWriter(ref hlog.GetKey(physicalAddress), ref v, ref hlog.GetValue(physicalAddress)); } status = OperationStatus.SUCCESS; @@ -1021,24 +1040,24 @@ internal OperationStatus InternalDelete( // Write default value // Ignore return value, the record is already marked Value v = default; - functions.ConcurrentWriter(ref hlog.GetKey(physicalAddress), ref v, ref hlog.GetValue(physicalAddress)); + fasterSession.ConcurrentWriter(ref hlog.GetKey(physicalAddress), ref v, ref hlog.GetValue(physicalAddress)); } status = OperationStatus.SUCCESS; goto LatchRelease; // Release shared latch (if acquired) } - // All other regions: Create a record in the mutable region - #endregion + // All other regions: Create a record in the mutable region + #endregion #region Create new record in the mutable region - CreateNewRecord: + CreateNewRecord: { var value = default(Value); // Immutable region or new record // Allocate default record size for tombstone var recordSize = hlog.GetRecordSize(ref key, ref value); - BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx); + BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx, fasterSession); var newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref hlog.GetInfo(newPhysicalAddress), sessionCtx.version, @@ -1072,7 +1091,7 @@ internal OperationStatus InternalDelete( #endregion #region Create pending context - CreatePendingContext: + CreatePendingContext: { pendingContext.type = OperationType.DELETE; pendingContext.key = hlog.GetKeyContainer(ref key); @@ -1085,7 +1104,7 @@ internal OperationStatus InternalDelete( #endregion #region Latch release - LatchRelease: + LatchRelease: { switch (latchOperation) { @@ -1103,7 +1122,7 @@ internal OperationStatus InternalDelete( if (status == OperationStatus.RETRY_NOW) { - return InternalDelete(ref key, ref userContext, ref pendingContext, sessionCtx, lsn); + return InternalDelete(ref key, ref userContext, ref pendingContext, fasterSession, sessionCtx, lsn); } else { @@ -1116,7 +1135,12 @@ internal OperationStatus InternalDelete( #region ContainsKeyInMemory [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status InternalContainsKeyInMemory(ref Key key, FasterExecutionContext sessionCtx, long fromAddress = -1) + internal Status InternalContainsKeyInMemory( + ref Key key, + FasterExecutionContext sessionCtx, + FasterSession fasterSession, + long fromAddress = -1) + where FasterSession : IFasterSession { if (fromAddress == -1) fromAddress = hlog.HeadAddress; @@ -1132,7 +1156,7 @@ internal Status InternalContainsKeyInMemory(ref Key key, FasterExecutionContext var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); if (sessionCtx.phase != Phase.REST) - HeavyEnter(hash, sessionCtx); + HeavyEnter(hash, sessionCtx, fasterSession); HashBucketEntry entry = default; var tagExists = FindTag(hash, tag, ref bucket, ref slot, ref entry); @@ -1182,6 +1206,7 @@ internal Status InternalContainsKeyInMemory(ref Key key, FasterExecutionContext /// The thread (or session) context to execute operation in. /// Async response from disk. /// Pending context corresponding to operation. + /// Callback functions. /// /// /// @@ -1195,10 +1220,13 @@ internal Status InternalContainsKeyInMemory(ref Key key, FasterExecutionContext /// /// /// - internal OperationStatus InternalContinuePendingRead( - FasterExecutionContext ctx, + internal OperationStatus InternalContinuePendingRead( + FasterExecutionContext ctx, AsyncIOContext request, - ref PendingContext pendingContext, FasterExecutionContext currentCtx) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext currentCtx) + where FasterSession : IFasterSession { Debug.Assert(RelaxedCPR || pendingContext.version == ctx.version); @@ -1209,12 +1237,12 @@ internal OperationStatus InternalContinuePendingRead( if (hlog.GetInfoFromBytePointer(request.record.GetValidPointer()).Tombstone) return OperationStatus.NOTFOUND; - functions.SingleReader(ref pendingContext.key.Get(), ref pendingContext.input, + fasterSession.SingleReader(ref pendingContext.key.Get(), ref pendingContext.input, ref hlog.GetContextRecordValue(ref request), ref pendingContext.output); if (CopyReadsToTail || UseReadCache) { - InternalContinuePendingReadCopyToTail(ctx, request, ref pendingContext, currentCtx); + InternalContinuePendingReadCopyToTail(ctx, request, ref pendingContext, fasterSession, currentCtx); } } else @@ -1229,11 +1257,15 @@ internal OperationStatus InternalContinuePendingRead( /// The thread(or session) context to execute operation in. /// Async response from disk. /// Pending context corresponding to operation. + /// Callback functions. /// - internal void InternalContinuePendingReadCopyToTail( - FasterExecutionContext opCtx, + internal void InternalContinuePendingReadCopyToTail( + FasterExecutionContext opCtx, AsyncIOContext request, - ref PendingContext pendingContext, FasterExecutionContext currentCtx) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext currentCtx) + where FasterSession : IFasterSession { Debug.Assert(RelaxedCPR || pendingContext.version == opCtx.version); @@ -1284,25 +1316,25 @@ internal void InternalContinuePendingReadCopyToTail( long newLogicalAddress, newPhysicalAddress; if (UseReadCache) { - BlockAllocateReadCache(recordSize, out newLogicalAddress, currentCtx); + BlockAllocateReadCache(recordSize, out newLogicalAddress, currentCtx, fasterSession); newPhysicalAddress = readcache.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref readcache.GetInfo(newPhysicalAddress), opCtx.version, true, false, false, entry.Address); readcache.ShallowCopy(ref pendingContext.key.Get(), ref readcache.GetKey(newPhysicalAddress)); - functions.SingleWriter(ref pendingContext.key.Get(), + fasterSession.SingleWriter(ref pendingContext.key.Get(), ref hlog.GetContextRecordValue(ref request), ref readcache.GetValue(newPhysicalAddress)); } else { - BlockAllocate(recordSize, out newLogicalAddress, currentCtx); + BlockAllocate(recordSize, out newLogicalAddress, currentCtx, fasterSession); newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref hlog.GetInfo(newPhysicalAddress), opCtx.version, true, false, false, latestLogicalAddress); hlog.ShallowCopy(ref pendingContext.key.Get(), ref hlog.GetKey(newPhysicalAddress)); - functions.SingleWriter(ref pendingContext.key.Get(), + fasterSession.SingleWriter(ref pendingContext.key.Get(), ref hlog.GetContextRecordValue(ref request), ref hlog.GetValue(newPhysicalAddress)); } @@ -1334,6 +1366,7 @@ ref hlog.GetContextRecordValue(ref request), /// thread (or session) context under which operation must be executed. /// record read from the disk. /// internal context for the pending RMW operation + /// Callback functions. /// Session context /// /// @@ -1355,10 +1388,13 @@ ref hlog.GetContextRecordValue(ref request), /// /// /// - internal OperationStatus InternalContinuePendingRMW( - FasterExecutionContext opCtx, + internal OperationStatus InternalContinuePendingRMW( + FasterExecutionContext opCtx, AsyncIOContext request, - ref PendingContext pendingContext, FasterExecutionContext sessionCtx) + ref PendingContext pendingContext, + FasterSession fasterSession, + FasterExecutionContext sessionCtx) + where FasterSession : IFasterSession { var recordSize = default(int); var bucket = default(HashBucket*); @@ -1411,9 +1447,9 @@ internal OperationStatus InternalContinuePendingRMW( else { physicalAddress = (long)request.record.GetValidPointer(); - recordSize = hlog.GetRecordSize(physicalAddress); + recordSize = hlog.GetRecordSize(physicalAddress, ref pendingContext.input); } - BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx); + BlockAllocate(recordSize, out long newLogicalAddress, sessionCtx, fasterSession); var newPhysicalAddress = hlog.GetPhysicalAddress(newLogicalAddress); RecordInfo.WriteInfo(ref hlog.GetInfo(newPhysicalAddress), opCtx.version, true, false, false, @@ -1421,14 +1457,14 @@ internal OperationStatus InternalContinuePendingRMW( hlog.ShallowCopy(ref key, ref hlog.GetKey(newPhysicalAddress)); if ((request.logicalAddress < hlog.BeginAddress) || (hlog.GetInfoFromBytePointer(request.record.GetValidPointer()).Tombstone)) { - functions.InitialUpdater(ref key, + fasterSession.InitialUpdater(ref key, ref pendingContext.input, ref hlog.GetValue(newPhysicalAddress)); status = OperationStatus.NOTFOUND; } else { - functions.CopyUpdater(ref key, + fasterSession.CopyUpdater(ref key, ref pendingContext.input, ref hlog.GetContextRecordValue(ref request), ref hlog.GetValue(newPhysicalAddress)); @@ -1458,7 +1494,7 @@ ref hlog.GetContextRecordValue(ref request), #endregion Retry: - return InternalRMW(ref pendingContext.key.Get(), ref pendingContext.input, ref pendingContext.userContext, ref pendingContext, sessionCtx, pendingContext.serialNum); + return InternalRMW(ref pendingContext.key.Get(), ref pendingContext.input, ref pendingContext.userContext, ref pendingContext, fasterSession, sessionCtx, pendingContext.serialNum); } #endregion @@ -1471,6 +1507,7 @@ ref hlog.GetContextRecordValue(ref request), /// Thread (or session) context under which operation was tried to execute. /// Current context /// Internal context of the operation. + /// Callback functions. /// Internal status of the trial. /// /// @@ -1488,16 +1525,18 @@ ref hlog.GetContextRecordValue(ref request), /// /// /// - internal Status HandleOperationStatus( - FasterExecutionContext opCtx, - FasterExecutionContext currentCtx, - PendingContext pendingContext, - OperationStatus status) + internal Status HandleOperationStatus( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + PendingContext pendingContext, + FasterSession fasterSession, + OperationStatus status) + where FasterSession : IFasterSession { if (status == OperationStatus.CPR_SHIFT_DETECTED) { #region Epoch Synchronization - SynchronizeEpoch(opCtx, currentCtx, ref pendingContext); + SynchronizeEpoch(opCtx, currentCtx, ref pendingContext, fasterSession); #endregion #region Retry as (v+1) Operation @@ -1509,24 +1548,24 @@ internal Status HandleOperationStatus( ref pendingContext.input, ref pendingContext.output, ref pendingContext.userContext, - ref pendingContext, currentCtx, pendingContext.serialNum); + ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.UPSERT: internalStatus = InternalUpsert(ref pendingContext.key.Get(), ref pendingContext.value.Get(), ref pendingContext.userContext, - ref pendingContext, currentCtx, pendingContext.serialNum); + ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.DELETE: internalStatus = InternalDelete(ref pendingContext.key.Get(), ref pendingContext.userContext, - ref pendingContext, currentCtx, pendingContext.serialNum); + ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.RMW: internalStatus = InternalRMW(ref pendingContext.key.Get(), ref pendingContext.input, ref pendingContext.userContext, - ref pendingContext, currentCtx, pendingContext.serialNum); + ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; } @@ -1570,26 +1609,32 @@ ref pendingContext.value.Get(), } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void SynchronizeEpoch(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, ref PendingContext pendingContext) + internal void SynchronizeEpoch( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + ref PendingContext pendingContext, + FasterSession fasterSession) + where FasterSession : IFasterSession { var version = opCtx.version; Debug.Assert(currentCtx.version == version); Debug.Assert(currentCtx.phase == Phase.PREPARE); - InternalRefresh(currentCtx); + InternalRefresh(currentCtx, fasterSession); Debug.Assert(currentCtx.version == version + 1); Debug.Assert(currentCtx.phase == Phase.IN_PROGRESS); pendingContext.version = currentCtx.version; } - internal AsyncIOContext ScheduleGetFromDisk(FasterExecutionContext opCtx, - ref PendingContext pendingContext) + internal AsyncIOContext ScheduleGetFromDisk( + FasterExecutionContext opCtx, + ref PendingContext pendingContext) { pendingContext.id = opCtx.totalPending++; - + // Issue asynchronous I/O request AsyncIOContext request = default; - + request.id = pendingContext.id; request.request_key = pendingContext.key; request.logicalAddress = pendingContext.logicalAddress; @@ -1625,15 +1670,16 @@ private void ReleaseSharedLatch(Key key) HashBucket.ReleaseSharedLatch(bucket); } - private void HeavyEnter(long hash, FasterExecutionContext ctx) + private void HeavyEnter(long hash, FasterExecutionContext ctx, FasterSession session) + where FasterSession : IFasterSession { if (ctx.phase == Phase.PREPARE_GROW) { // We spin-wait as a simplification // Could instead do a "heavy operation" here - while (_systemState.phase != Phase.IN_PROGRESS_GROW) + while (systemState.phase != Phase.IN_PROGRESS_GROW) Thread.SpinWait(100); - InternalRefresh(ctx); + InternalRefresh(ctx, session); } if (ctx.phase == Phase.IN_PROGRESS_GROW) { @@ -1642,21 +1688,31 @@ private void HeavyEnter(long hash, FasterExecutionContext ctx) } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void BlockAllocate(int recordSize, out long logicalAddress, FasterExecutionContext ctx) + private void BlockAllocate( + int recordSize, + out long logicalAddress, + FasterExecutionContext ctx, + FasterSession fasterSession) + where FasterSession : IFasterSession { while ((logicalAddress = hlog.TryAllocate(recordSize)) == 0) { - InternalRefresh(ctx); + InternalRefresh(ctx, fasterSession); Thread.Yield(); } } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private void BlockAllocateReadCache(int recordSize, out long logicalAddress, FasterExecutionContext currentCtx) + private void BlockAllocateReadCache( + int recordSize, + out long logicalAddress, + FasterExecutionContext currentCtx, + FasterSession fasterSession) + where FasterSession : IFasterSession { while ((logicalAddress = readcache.TryAllocate(recordSize)) == 0) { - InternalRefresh(currentCtx); + InternalRefresh(currentCtx, fasterSession); Thread.Yield(); } } @@ -1724,7 +1780,7 @@ private void SplitBuckets(long hash) { // GC old version of hash table state[1 - resizeInfo.version] = default; - GlobalStateMachineStep(_systemState); + GlobalStateMachineStep(systemState); return; } break; diff --git a/cs/src/core/Index/FASTER/FASTERLegacy.cs b/cs/src/core/Index/FASTER/FASTERLegacy.cs index fbf4a1b16..ceec88e5f 100644 --- a/cs/src/core/Index/FASTER/FASTERLegacy.cs +++ b/cs/src/core/Index/FASTER/FASTERLegacy.cs @@ -5,30 +5,71 @@ using System; using System.Diagnostics; +using System.Threading; +using System.Threading.Tasks; namespace FASTER.core { - public partial class FasterKV : FasterBase, IFasterKV + /// + /// The legacy FASTER key-value store + /// + /// Key + /// Value + /// Input + /// Output + /// Context + /// Functions + //[Obsolete("Use FasteKV that provides functions with sessions")] + public partial class FasterKV : IDisposable, IFasterKV where Key : new() where Value : new() where Functions : IFunctions { - private FastThreadLocal threadCtx; + private FastThreadLocal.FasterExecutionContext> _threadCtx; + + private readonly FasterKV _fasterKV; + private readonly Functions _functions; + + private LegacyFasterSession FasterSession => new LegacyFasterSession(this); + + /// + public long EntryCount => _fasterKV.EntryCount; + + /// + public long IndexSize => _fasterKV.IndexSize; + + /// + public IFasterEqualityComparer Comparer => _fasterKV.Comparer; + + /// + public LogAccessor Log => _fasterKV.Log; + + /// + public LogAccessor ReadCache => _fasterKV.ReadCache; + + /// + public FasterKV(long size, Functions functions, LogSettings logSettings, + CheckpointSettings checkpointSettings = null, SerializerSettings serializerSettings = null, + IFasterEqualityComparer comparer = null, + VariableLengthStructSettings variableLengthStructSettings = null) + { + _functions = functions; + _fasterKV = new FasterKV(size, logSettings, checkpointSettings, serializerSettings, comparer, variableLengthStructSettings); + } /// /// Dispose FASTER instance - legacy items /// private void LegacyDispose() { - threadCtx?.Dispose(); + _threadCtx?.Dispose(); } private bool InLegacySession() { - return threadCtx != null; + return _threadCtx != null; } - /// /// Legacy API: Start session with FASTER - call once per thread before using FASTER /// @@ -36,8 +77,8 @@ private bool InLegacySession() [Obsolete("Use NewSession() instead.")] public Guid StartSession() { - if (threadCtx == null) - threadCtx = new FastThreadLocal(); + if (_threadCtx == null) + _threadCtx = new FastThreadLocal.FasterExecutionContext>(); return InternalAcquire(); } @@ -52,8 +93,8 @@ public CommitPoint ContinueSession(Guid guid) { StartSession(); - var cp = InternalContinue(guid.ToString(), out FasterExecutionContext ctx); - threadCtx.Value = ctx; + var cp = _fasterKV.InternalContinue(guid.ToString(), out var ctx); + _threadCtx.Value = ctx; return cp; } @@ -64,7 +105,7 @@ public CommitPoint ContinueSession(Guid guid) [Obsolete("Use and dispose NewSession() instead.")] public void StopSession() { - InternalRelease(this.threadCtx.Value); + InternalRelease(_threadCtx.Value); } /// @@ -73,10 +114,9 @@ public void StopSession() [Obsolete("Use NewSession(), where Refresh() is not required by default.")] public void Refresh() { - InternalRefresh(threadCtx.Value); + _fasterKV.InternalRefresh(_threadCtx.Value, FasterSession); } - /// /// Legacy API: Complete all pending operations issued by this session /// @@ -85,7 +125,7 @@ public void Refresh() [Obsolete("Use NewSession() and invoke CompletePending() on the session.")] public bool CompletePending(bool wait = false) { - return InternalCompletePending(threadCtx.Value, wait); + return _fasterKV.InternalCompletePending(_threadCtx.Value, FasterSession, wait); } /// @@ -100,7 +140,7 @@ public bool CompletePending(bool wait = false) [Obsolete("Use NewSession() and invoke Read() on the session.")] public Status Read(ref Key key, ref Input input, ref Output output, Context context, long serialNo) { - return ContextRead(ref key, ref input, ref output, context, serialNo, threadCtx.Value); + return _fasterKV.ContextRead(ref key, ref input, ref output, context, FasterSession, serialNo, _threadCtx.Value); } /// @@ -114,7 +154,7 @@ public Status Read(ref Key key, ref Input input, ref Output output, Context cont [Obsolete("Use NewSession() and invoke Upsert() on the session.")] public Status Upsert(ref Key key, ref Value value, Context context, long serialNo) { - return ContextUpsert(ref key, ref value, context, serialNo, threadCtx.Value); + return _fasterKV.ContextUpsert(ref key, ref value, context, FasterSession, serialNo, _threadCtx.Value); } /// @@ -128,7 +168,7 @@ public Status Upsert(ref Key key, ref Value value, Context context, long serialN [Obsolete("Use NewSession() and invoke RMW() on the session.")] public Status RMW(ref Key key, ref Input input, Context context, long serialNo) { - return ContextRMW(ref key, ref input, context, serialNo, threadCtx.Value); + return _fasterKV.ContextRMW(ref key, ref input, context, FasterSession, serialNo, _threadCtx.Value); } /// @@ -144,7 +184,7 @@ public Status RMW(ref Key key, ref Input input, Context context, long serialNo) [Obsolete("Use NewSession() and invoke Delete() on the session.")] public Status Delete(ref Key key, Context context, long serialNo) { - return ContextDelete(ref key, context, serialNo, threadCtx.Value); + return _fasterKV.ContextDelete(ref key, context, FasterSession, serialNo, _threadCtx.Value); } /// @@ -157,7 +197,7 @@ public bool CompleteCheckpoint(bool spinWait = false) { if (!InLegacySession()) { - CompleteCheckpointAsync().GetAwaiter().GetResult(); + _fasterKV.CompleteCheckpointAsync().GetAwaiter().GetResult(); return true; } @@ -168,7 +208,7 @@ public bool CompleteCheckpoint(bool spinWait = false) do { CompletePending(); - if (_systemState.phase == Phase.REST) + if (_fasterKV.systemState.phase == Phase.REST) { CompletePending(); return true; @@ -180,25 +220,25 @@ public bool CompleteCheckpoint(bool spinWait = false) private Guid InternalAcquire() { - epoch.Resume(); - threadCtx.InitializeThread(); - Phase phase = _systemState.phase; + _fasterKV.epoch.Resume(); + _threadCtx.InitializeThread(); + Phase phase = _fasterKV.systemState.phase; if (phase != Phase.REST) { throw new FasterException("Can acquire only in REST phase!"); } Guid guid = Guid.NewGuid(); - threadCtx.Value = new FasterExecutionContext(); - InitContext(threadCtx.Value, guid.ToString()); + _threadCtx.Value = new FasterKV.FasterExecutionContext(); + _fasterKV.InitContext(_threadCtx.Value, guid.ToString()); - threadCtx.Value.prevCtx = new FasterExecutionContext(); - InitContext(threadCtx.Value.prevCtx, guid.ToString()); - threadCtx.Value.prevCtx.version--; - InternalRefresh(threadCtx.Value); + _threadCtx.Value.prevCtx = new FasterKV.FasterExecutionContext(); + _fasterKV.InitContext(_threadCtx.Value.prevCtx, guid.ToString()); + _threadCtx.Value.prevCtx.version--; + _fasterKV.InternalRefresh(_threadCtx.Value, FasterSession); return guid; } - private void InternalRelease(FasterExecutionContext ctx) + private void InternalRelease(FasterKV.FasterExecutionContext ctx) { Debug.Assert(ctx.HasNoPendingRequests); if (ctx.prevCtx != null) @@ -207,7 +247,128 @@ private void InternalRelease(FasterExecutionContext ctx) } Debug.Assert(ctx.phase == Phase.REST); - epoch.Suspend(); + _fasterKV.epoch.Suspend(); + } + + /// + public void Dispose() + { + _fasterKV.Dispose(); + LegacyDispose(); + } + + /// + public ClientSession NewSession(string sessionId = null, bool threadAffinitized = false) + => _fasterKV.NewSession(_functions, sessionId, threadAffinitized); + + /// + public ClientSession ResumeSession(string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + => _fasterKV.ResumeSession(_functions, sessionId, out commitPoint, threadAffinitized); + + /// + public bool GrowIndex() => _fasterKV.GrowIndex(); + + /// + public bool TakeFullCheckpoint(out Guid token, long targetVersion = -1) => _fasterKV.TakeFullCheckpoint(out token, targetVersion); + + /// + public bool TakeIndexCheckpoint(out Guid token) => _fasterKV.TakeIndexCheckpoint(out token); + + /// + public bool TakeHybridLogCheckpoint(out Guid token, long targetVersion = -1) => _fasterKV.TakeHybridLogCheckpoint(out token, targetVersion); + + /// + public void Recover() => _fasterKV.Recover(); + + /// + public void Recover(Guid fullcheckpointToken) => _fasterKV.Recover(fullcheckpointToken); + + /// + public void Recover(Guid indexToken, Guid hybridLogToken) => _fasterKV.Recover(indexToken, hybridLogToken); + + /// + public ValueTask CompleteCheckpointAsync(CancellationToken token = default(CancellationToken)) => _fasterKV.CompleteCheckpointAsync(token); + + /// + public string DumpDistribution() => _fasterKV.DumpDistribution(); + + // This is a struct to allow JIT to inline calls (and bypass default interface call mechanism) + private struct LegacyFasterSession : IFasterSession + { + private readonly FasterKV _fasterKV; + + public LegacyFasterSession(FasterKV fasterKV) + { + _fasterKV = fasterKV; + } + + public void CheckpointCompletionCallback(string guid, CommitPoint commitPoint) + { + _fasterKV._functions.CheckpointCompletionCallback(guid, commitPoint); + } + + public void ConcurrentReader(ref Key key, ref Input input, ref Value value, ref Output dst) + { + _fasterKV._functions.ConcurrentReader(ref key, ref input, ref value, ref dst); + } + + public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) + { + return _fasterKV._functions.ConcurrentWriter(ref key, ref src, ref dst); + } + + public void CopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue) + { + _fasterKV._functions.CopyUpdater(ref key, ref input, ref oldValue, ref newValue); + } + + public void DeleteCompletionCallback(ref Key key, Context ctx) + { + _fasterKV._functions.DeleteCompletionCallback(ref key, ctx); + } + + public void InitialUpdater(ref Key key, ref Input input, ref Value value) + { + _fasterKV._functions.InitialUpdater(ref key, ref input, ref value); + } + + public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value) + { + return _fasterKV._functions.InPlaceUpdater(ref key, ref input, ref value); + } + + public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, Context ctx, Status status) + { + _fasterKV._functions.ReadCompletionCallback(ref key, ref input, ref output, ctx, status); + } + + public void RMWCompletionCallback(ref Key key, ref Input input, Context ctx, Status status) + { + _fasterKV._functions.RMWCompletionCallback(ref key, ref input, ctx, status); + } + + public void SingleReader(ref Key key, ref Input input, ref Value value, ref Output dst) + { + _fasterKV._functions.SingleReader(ref key, ref input, ref value, ref dst); + } + + public void SingleWriter(ref Key key, ref Value src, ref Value dst) + { + _fasterKV._functions.SingleWriter(ref key, ref src, ref dst); + } + + public void UnsafeResumeThread() + { + } + + public void UnsafeSuspendThread() + { + } + + public void UpsertCompletionCallback(ref Key key, ref Value value, Context ctx) + { + _fasterKV._functions.UpsertCompletionCallback(ref key, ref value, ctx); + } } } } diff --git a/cs/src/core/Index/FASTER/FASTERThread.cs b/cs/src/core/Index/FASTER/FASTERThread.cs index 038bc5763..fa6731540 100644 --- a/cs/src/core/Index/FASTER/FASTERThread.cs +++ b/cs/src/core/Index/FASTER/FASTERThread.cs @@ -10,12 +10,11 @@ namespace FASTER.core { - public partial class FasterKV : FasterBase, IFasterKV + public partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { - internal CommitPoint InternalContinue(string guid, out FasterExecutionContext ctx) + internal CommitPoint InternalContinue(string guid, out FasterExecutionContext ctx) { ctx = null; @@ -25,7 +24,7 @@ internal CommitPoint InternalContinue(string guid, out FasterExecutionContext ct { // We have recovered the corresponding session. // Now obtain the session by first locking the rest phase - var currentState = SystemState.Copy(ref _systemState); + var currentState = SystemState.Copy(ref systemState); if (currentState.phase == Phase.REST) { var intermediateState = SystemState.Make(Phase.INTERMEDIATE, currentState.version); @@ -36,9 +35,9 @@ internal CommitPoint InternalContinue(string guid, out FasterExecutionContext ct { // We have atomically removed session details. // No one else can continue this session - ctx = new FasterExecutionContext(); + ctx = new FasterExecutionContext(); InitContext(ctx, guid); - ctx.prevCtx = new FasterExecutionContext(); + ctx.prevCtx = new FasterExecutionContext(); InitContext(ctx.prevCtx, guid); ctx.prevCtx.version--; ctx.serialNum = cp.UntilSerialNo; @@ -66,29 +65,29 @@ internal CommitPoint InternalContinue(string guid, out FasterExecutionContext ct } [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void InternalRefresh(FasterExecutionContext ctx, ClientSession clientSession = null) + internal void InternalRefresh(FasterExecutionContext ctx, FasterSession fasterSession) + where FasterSession : IFasterSession { epoch.ProtectAndDrain(); // We check if we are in normal mode - var newPhaseInfo = SystemState.Copy(ref _systemState); + var newPhaseInfo = SystemState.Copy(ref systemState); if (ctx.phase == Phase.REST && newPhaseInfo.phase == Phase.REST && ctx.version == newPhaseInfo.version) { return; } - // await is never invoked when calling the function with async = false - #pragma warning disable 4014 - ThreadStateMachineStep(ctx, clientSession, false); - #pragma warning restore 4014 +#pragma warning disable 4014 + var task = ThreadStateMachineStep(ctx, fasterSession, false); + Debug.Assert(task.IsCompleted); +#pragma warning restore 4014 } - - internal void InitContext(FasterExecutionContext ctx, string token, long lsn = -1) + internal void InitContext(FasterExecutionContext ctx, string token, long lsn = -1) { ctx.phase = Phase.REST; - ctx.version = _systemState.version; + ctx.version = systemState.version; ctx.markers = new bool[8]; ctx.serialNum = lsn; ctx.guid = token; @@ -97,23 +96,23 @@ internal void InitContext(FasterExecutionContext ctx, string token, long lsn = - { if (ctx.retryRequests == null) { - ctx.retryRequests = new Queue(); + ctx.retryRequests = new Queue>(); ctx.readyResponses = new AsyncQueue>(); - ctx.ioPendingRequests = new Dictionary(); + ctx.ioPendingRequests = new Dictionary>(); ctx.pendingReads = new AsyncCountDown(); } } else { ctx.totalPending = 0; - ctx.retryRequests = new Queue(); + ctx.retryRequests = new Queue>(); ctx.readyResponses = new AsyncQueue>(); - ctx.ioPendingRequests = new Dictionary(); + ctx.ioPendingRequests = new Dictionary>(); ctx.pendingReads = new AsyncCountDown(); } } - internal void CopyContext(FasterExecutionContext src, FasterExecutionContext dst) + internal void CopyContext(FasterExecutionContext src, FasterExecutionContext dst) { dst.phase = src.phase; dst.version = src.version; @@ -143,7 +142,11 @@ internal void CopyContext(FasterExecutionContext src, FasterExecutionContext dst } } - internal bool InternalCompletePending(FasterExecutionContext ctx, bool wait = false) + internal bool InternalCompletePending( + FasterExecutionContext ctx, + FasterSession fasterSession, + bool wait = false) + where FasterSession : IFasterSession { do { @@ -154,18 +157,18 @@ internal bool InternalCompletePending(FasterExecutionContext ctx, bool wait = fa { if (ctx.phase == Phase.IN_PROGRESS || ctx.phase == Phase.WAIT_PENDING) { - InternalCompletePendingRequests(ctx.prevCtx, ctx); - InternalCompleteRetryRequests(ctx.prevCtx, ctx); - InternalRefresh(ctx); + InternalCompletePendingRequests(ctx.prevCtx, ctx, fasterSession); + InternalCompleteRetryRequests(ctx.prevCtx, ctx, fasterSession); + InternalRefresh(ctx, fasterSession); done &= (ctx.prevCtx.HasNoPendingRequests); } } #endregion - InternalCompletePendingRequests(ctx, ctx); - InternalCompleteRetryRequests(ctx, ctx); - InternalRefresh(ctx); + InternalCompletePendingRequests(ctx, ctx, fasterSession); + InternalCompleteRetryRequests(ctx, ctx, fasterSession); + InternalRefresh(ctx, fasterSession); done &= (ctx.HasNoPendingRequests); @@ -184,25 +187,34 @@ internal bool InternalCompletePending(FasterExecutionContext ctx, bool wait = fa return false; } - internal bool InRestPhase() => _systemState.phase == Phase.REST; + internal bool InRestPhase() => systemState.phase == Phase.REST; #region Complete Retry Requests - internal void InternalCompleteRetryRequests(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, ClientSession clientSession = null) + internal void InternalCompleteRetryRequests( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + FasterSession fasterSession) + where FasterSession : IFasterSession { int count = opCtx.retryRequests.Count; if (count == 0) return; - clientSession?.UnsafeResumeThread(); + fasterSession.UnsafeResumeThread(); for (int i = 0; i < count; i++) { var pendingContext = opCtx.retryRequests.Dequeue(); - InternalCompleteRetryRequest(opCtx, currentCtx, pendingContext); + InternalCompleteRetryRequest(opCtx, currentCtx, pendingContext, fasterSession); } - clientSession?.UnsafeSuspendThread(); + fasterSession.UnsafeSuspendThread(); } - internal void InternalCompleteRetryRequest(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, PendingContext pendingContext) + internal void InternalCompleteRetryRequest( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + PendingContext pendingContext, + FasterSession fasterSession) + where FasterSession : IFasterSession { var internalStatus = default(OperationStatus); ref Key key = ref pendingContext.key.Get(); @@ -212,13 +224,13 @@ internal void InternalCompleteRetryRequest(FasterExecutionContext opCtx, FasterE switch (pendingContext.type) { case OperationType.RMW: - internalStatus = InternalRMW(ref key, ref pendingContext.input, ref pendingContext.userContext, ref pendingContext, currentCtx, pendingContext.serialNum); + internalStatus = InternalRMW(ref key, ref pendingContext.input, ref pendingContext.userContext, ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.UPSERT: - internalStatus = InternalUpsert(ref key, ref value, ref pendingContext.userContext, ref pendingContext, currentCtx, pendingContext.serialNum); + internalStatus = InternalUpsert(ref key, ref value, ref pendingContext.userContext, ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.DELETE: - internalStatus = InternalDelete(ref key, ref pendingContext.userContext, ref pendingContext, currentCtx, pendingContext.serialNum); + internalStatus = InternalDelete(ref key, ref pendingContext.userContext, ref pendingContext, fasterSession, currentCtx, pendingContext.serialNum); break; case OperationType.READ: throw new FasterException("Cannot happen!"); @@ -233,7 +245,7 @@ internal void InternalCompleteRetryRequest(FasterExecutionContext opCtx, FasterE } else { - status = HandleOperationStatus(opCtx, currentCtx, pendingContext, internalStatus); + status = HandleOperationStatus(opCtx, currentCtx, pendingContext, fasterSession, internalStatus); } // If done, callback user code. @@ -245,17 +257,17 @@ internal void InternalCompleteRetryRequest(FasterExecutionContext opCtx, FasterE switch (pendingContext.type) { case OperationType.RMW: - functions.RMWCompletionCallback(ref key, + fasterSession.RMWCompletionCallback(ref key, ref pendingContext.input, pendingContext.userContext, status); break; case OperationType.UPSERT: - functions.UpsertCompletionCallback(ref key, + fasterSession.UpsertCompletionCallback(ref key, ref value, pendingContext.userContext); break; case OperationType.DELETE: - functions.DeleteCompletionCallback(ref key, + fasterSession.DeleteCompletionCallback(ref key, pendingContext.userContext); break; default: @@ -267,17 +279,26 @@ internal void InternalCompleteRetryRequest(FasterExecutionContext opCtx, FasterE #endregion #region Complete Pending Requests - internal void InternalCompletePendingRequests(FasterExecutionContext opCtx, FasterExecutionContext currentCtx) + internal void InternalCompletePendingRequests( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + FasterSession fasterSession) + where FasterSession : IFasterSession { if (opCtx.readyResponses.Count == 0) return; while (opCtx.readyResponses.TryDequeue(out AsyncIOContext request)) { - InternalCompletePendingRequest(opCtx, currentCtx, request); + InternalCompletePendingRequest(opCtx, currentCtx, fasterSession, request); } } - internal async ValueTask InternalCompletePendingRequestsAsync(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, ClientSession clientSession, CancellationToken token = default) + internal async ValueTask InternalCompletePendingRequestsAsync( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + FasterSession fasterSession, + CancellationToken token = default) + where FasterSession : IFasterSession { while (opCtx.SyncIoPendingCount > 0) { @@ -285,28 +306,33 @@ internal async ValueTask InternalCompletePendingRequestsAsync(FasterExecutionCon if (opCtx.readyResponses.Count > 0) { - clientSession.UnsafeResumeThread(); + fasterSession.UnsafeResumeThread(); while (opCtx.readyResponses.Count > 0) { opCtx.readyResponses.TryDequeue(out request); - InternalCompletePendingRequest(opCtx, currentCtx, request); + InternalCompletePendingRequest(opCtx, currentCtx, fasterSession, request); } - clientSession.UnsafeSuspendThread(); + fasterSession.UnsafeSuspendThread(); } else { request = await opCtx.readyResponses.DequeueAsync(token); - clientSession.UnsafeResumeThread(); - InternalCompletePendingRequest(opCtx, currentCtx, request); - clientSession.UnsafeSuspendThread(); + fasterSession.UnsafeResumeThread(); + InternalCompletePendingRequest(opCtx, currentCtx, fasterSession, request); + fasterSession.UnsafeSuspendThread(); } } } - internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, AsyncIOContext request) + internal void InternalCompletePendingRequest( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + FasterSession fasterSession, + AsyncIOContext request) + where FasterSession : IFasterSession { - if (opCtx.ioPendingRequests.TryGetValue(request.id, out PendingContext pendingContext)) + if (opCtx.ioPendingRequests.TryGetValue(request.id, out var pendingContext)) { ref Key key = ref pendingContext.key.Get(); @@ -317,11 +343,11 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste // Issue the continue command if (pendingContext.type == OperationType.READ) { - internalStatus = InternalContinuePendingRead(opCtx, request, ref pendingContext, currentCtx); + internalStatus = InternalContinuePendingRead(opCtx, request, ref pendingContext, fasterSession, currentCtx); } else { - internalStatus = InternalContinuePendingRMW(opCtx, request, ref pendingContext, currentCtx); ; + internalStatus = InternalContinuePendingRMW(opCtx, request, ref pendingContext, fasterSession, currentCtx); } request.Dispose(); @@ -334,7 +360,7 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste } else { - status = HandleOperationStatus(opCtx, currentCtx, pendingContext, internalStatus); + status = HandleOperationStatus(opCtx, currentCtx, pendingContext, fasterSession, internalStatus); } // If done, callback user code @@ -345,7 +371,7 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste if (pendingContext.type == OperationType.READ) { - functions.ReadCompletionCallback(ref key, + fasterSession.ReadCompletionCallback(ref key, ref pendingContext.input, ref pendingContext.output, pendingContext.userContext, @@ -353,7 +379,7 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste } else { - functions.RMWCompletionCallback(ref key, + fasterSession.RMWCompletionCallback(ref key, ref pendingContext.input, pendingContext.userContext, status); @@ -363,13 +389,19 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste } } - internal (Status, Output) InternalCompletePendingReadRequest(FasterExecutionContext opCtx, FasterExecutionContext currentCtx, AsyncIOContext request, PendingContext pendingContext) + internal (Status, Output) InternalCompletePendingReadRequest( + FasterExecutionContext opCtx, + FasterExecutionContext currentCtx, + FasterSession fasterSession, + AsyncIOContext request, + PendingContext pendingContext) + where FasterSession : IFasterSession { (Status, Output) s = default; ref Key key = ref pendingContext.key.Get(); - OperationStatus internalStatus = InternalContinuePendingRead(opCtx, request, ref pendingContext, currentCtx); + OperationStatus internalStatus = InternalContinuePendingRead(opCtx, request, ref pendingContext, fasterSession, currentCtx); request.Dispose(); @@ -387,7 +419,7 @@ internal void InternalCompletePendingRequest(FasterExecutionContext opCtx, Faste if (pendingContext.heldLatch == LatchOperation.Shared) ReleaseSharedLatch(key); - functions.ReadCompletionCallback(ref key, + fasterSession.ReadCompletionCallback(ref key, ref pendingContext.input, ref pendingContext.output, pendingContext.userContext, diff --git a/cs/src/core/Index/FASTER/LogAccessor.cs b/cs/src/core/Index/FASTER/LogAccessor.cs index f73a57a2f..deb248ab8 100644 --- a/cs/src/core/Index/FASTER/LogAccessor.cs +++ b/cs/src/core/Index/FASTER/LogAccessor.cs @@ -4,6 +4,7 @@ #pragma warning disable 0162 using System; +using System.Runtime.CompilerServices; namespace FASTER.core { @@ -12,16 +13,11 @@ namespace FASTER.core /// /// /// - /// - /// - /// - /// - public sealed class LogAccessor : IObservable> + public sealed class LogAccessor : IObservable> where Key : new() where Value : new() - where Functions : IFunctions { - private readonly FasterKV fht; + private readonly FasterKV fht; private readonly AllocatorBase allocator; /// @@ -29,7 +25,7 @@ public sealed class LogAccessor : /// /// /// - public LogAccessor(FasterKV fht, AllocatorBase allocator) + public LogAccessor(FasterKV fht, AllocatorBase allocator) { this.fht = fht; this.allocator = allocator; @@ -179,149 +175,245 @@ public void DisposeFromMemory() /// /// Compact the log until specified address, moving active - /// records to the tail of the log + /// records to the tail of the log. + /// Uses default compaction functions that only deletes explicitly deleted records, copying is implemeted by shallow copying values from source to destination. /// /// public void Compact(long untilAddress) { if (allocator is VariableLengthBlittableAllocator varLen) { - var functions = new LogVariableCompactFunctions(varLen); + var functions = new LogVariableCompactFunctions(varLen, default(DefaultVariableCompactionFunctions)); var variableLengthStructSettings = new VariableLengthStructSettings { keyLength = varLen.KeyLength, valueLength = varLen.ValueLength, }; - Compact(functions, untilAddress, variableLengthStructSettings); + Compact(functions, default(DefaultVariableCompactionFunctions), untilAddress, variableLengthStructSettings); } else { - Compact(new LogCompactFunctions(), untilAddress, null); + Compact(new LogCompactFunctions(default(DefaultCompactionFunctions)), default(DefaultCompactionFunctions), untilAddress, null); } } - private void Compact(T functions, long untilAddress, VariableLengthStructSettings variableLengthStructSettings) - where T : IFunctions + /// + /// Compact the log until specified address, moving active + /// records to the tail of the log. + /// + /// User provided compaction functions (see ). + /// + public void Compact(CompactionFunctions compactionFunctions, long untilAddress) + where CompactionFunctions : ICompactionFunctions { - var fhtSession = fht.NewSession(); - - var originalUntilAddress = untilAddress; - - var tempKv = new FasterKV - (fht.IndexSize, functions, new LogSettings(), comparer: fht.Comparer, variableLengthStructSettings: variableLengthStructSettings); - var tempKvSession = tempKv.NewSession(); - - using (var iter1 = fht.Log.Scan(fht.Log.BeginAddress, untilAddress)) + if (allocator is VariableLengthBlittableAllocator varLen) { - while (iter1.GetNext(out RecordInfo recordInfo)) + var functions = new LogVariableCompactFunctions(varLen, compactionFunctions); + var variableLengthStructSettings = new VariableLengthStructSettings { - ref var key = ref iter1.GetKey(); - ref var value = ref iter1.GetValue(); + keyLength = varLen.KeyLength, + valueLength = varLen.ValueLength, + }; - if (recordInfo.Tombstone) - tempKvSession.Delete(ref key, default, 0); - else - tempKvSession.Upsert(ref key, ref value, default, 0); - } + Compact(functions, compactionFunctions, untilAddress, variableLengthStructSettings); } + else + { + Compact(new LogCompactFunctions(compactionFunctions), compactionFunctions, untilAddress, null); + } + } - // TODO: Scan until SafeReadOnlyAddress - long scanUntil = untilAddress; - LogScanForValidity(ref untilAddress, ref scanUntil, ref tempKvSession); - - // Make sure key wasn't inserted between SafeReadOnlyAddress and TailAddress + private unsafe void Compact(Functions functions, CompactionFunctions cf, long untilAddress, VariableLengthStructSettings variableLengthStructSettings) + where Functions : IFunctions + where CompactionFunctions : ICompactionFunctions + { + var originalUntilAddress = untilAddress; - using (var iter3 = tempKv.Log.Scan(tempKv.Log.BeginAddress, tempKv.Log.TailAddress)) + using (var fhtSession = fht.NewSession(functions)) + using (var tempKv = new FasterKV(fht.IndexSize, new LogSettings(), comparer: fht.Comparer, variableLengthStructSettings: variableLengthStructSettings)) + using (var tempKvSession = tempKv.NewSession(functions)) { - while (iter3.GetNext(out RecordInfo recordInfo)) + using (var iter1 = fht.Log.Scan(fht.Log.BeginAddress, untilAddress)) { - ref var key = ref iter3.GetKey(); - ref var value = ref iter3.GetValue(); - - if (!recordInfo.Tombstone) + while (iter1.GetNext(out var recordInfo)) { - if (fhtSession.ContainsKeyInMemory(ref key, scanUntil) == Status.NOTFOUND) - fhtSession.Upsert(ref key, ref value, default, 0); + ref var key = ref iter1.GetKey(); + ref var value = ref iter1.GetValue(); + + if (recordInfo.Tombstone || cf.IsDeleted(key, value)) + tempKvSession.Delete(ref key, default, 0); + else + tempKvSession.Upsert(ref key, ref value, default, 0); } - if (scanUntil < fht.Log.SafeReadOnlyAddress) + } + + // TODO: Scan until SafeReadOnlyAddress + var scanUntil = untilAddress; + LogScanForValidity(ref untilAddress, ref scanUntil, tempKvSession); + + // Make sure key wasn't inserted between SafeReadOnlyAddress and TailAddress + using (var iter3 = tempKv.Log.Scan(tempKv.Log.BeginAddress, tempKv.Log.TailAddress)) + { + while (iter3.GetNext(out var recordInfo)) { - LogScanForValidity(ref untilAddress, ref scanUntil, ref tempKvSession); + ref var key = ref iter3.GetKey(); + ref var value = ref iter3.GetValue(); + + if (!recordInfo.Tombstone) + { + if (fhtSession.ContainsKeyInMemory(ref key, scanUntil) == Status.NOTFOUND) + { + // Check if recordInfo point to the newest record. + // With #164 it is possible that tempKv might have multiple records with the same + // key (ConcurrentWriter returns false). For this reason check the index + // whether the actual record has the same address (or maybe even deleted). + // If this is too much of a performance hit - we could try and add additional info + // to the recordInfo to indicate that it was replaced (but it would only for tempKv + // not general case). + var bucket = default(HashBucket*); + var slot = default(int); + + var hash = tempKv.Comparer.GetHashCode64(ref key); + var tag = (ushort)((ulong)hash >> Constants.kHashTagShift); + + var entry = default(HashBucketEntry); + if (tempKv.FindTag(hash, tag, ref bucket, ref slot, ref entry) && entry.Address == iter3.CurrentAddress) + fhtSession.Upsert(ref key, ref value, default, 0); + } + } + if (scanUntil < fht.Log.SafeReadOnlyAddress) + { + LogScanForValidity(ref untilAddress, ref scanUntil, tempKvSession); + } } } } - fhtSession.Dispose(); - tempKvSession.Dispose(); - tempKv.Dispose(); ShiftBeginAddress(originalUntilAddress); } - private void LogScanForValidity(ref long untilAddress, ref long scanUntil, ref ClientSession tempKvSession) - where T : IFunctions + private void LogScanForValidity(ref long untilAddress, ref long scanUntil, ClientSession tempKvSession) + where Functions : IFunctions { while (scanUntil < fht.Log.SafeReadOnlyAddress) { untilAddress = scanUntil; scanUntil = fht.Log.SafeReadOnlyAddress; - using var iter2 = fht.Log.Scan(untilAddress, scanUntil); - while (iter2.GetNext(out RecordInfo _)) + using (var iter2 = fht.Log.Scan(untilAddress, scanUntil)) { - ref var key = ref iter2.GetKey(); - ref var value = ref iter2.GetValue(); + while (iter2.GetNext(out var _)) + { + ref var key = ref iter2.GetKey(); + ref var value = ref iter2.GetValue(); - tempKvSession.Delete(ref key, default, 0); + tempKvSession.Delete(ref key, default, 0); + } } } } - private sealed class LogVariableCompactFunctions : IFunctions + private sealed class LogVariableCompactFunctions : IFunctions + where CompactionFunctions : ICompactionFunctions { - private readonly VariableLengthBlittableAllocator allocator; + private readonly VariableLengthBlittableAllocator _allocator; + private readonly CompactionFunctions _functions; - public LogVariableCompactFunctions(VariableLengthBlittableAllocator allocator) + public LogVariableCompactFunctions(VariableLengthBlittableAllocator allocator, CompactionFunctions functions) { - this.allocator = allocator; + _allocator = allocator; + _functions = functions; } public void CheckpointCompletionCallback(string sessionId, CommitPoint commitPoint) { } - public void ConcurrentReader(ref Key key, ref Input input, ref Value value, ref Output dst) { } - public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) + public void ConcurrentReader(ref Key key, ref Empty input, ref Value value, ref Empty dst) { } + public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) { return _functions.CopyInPlace(ref src, ref dst, _allocator.ValueLength); } + public void CopyUpdater(ref Key key, ref Empty input, ref Value oldValue, ref Value newValue) { } + public void InitialUpdater(ref Key key, ref Empty input, ref Value value) { } + public bool InPlaceUpdater(ref Key key, ref Empty input, ref Value value) => false; + public void ReadCompletionCallback(ref Key key, ref Empty input, ref Empty output, Empty ctx, Status status) { } + public void RMWCompletionCallback(ref Key key, ref Empty input, Empty ctx, Status status) { } + public void SingleReader(ref Key key, ref Empty input, ref Value value, ref Empty dst) { } + public void SingleWriter(ref Key key, ref Value src, ref Value dst) { _functions.Copy(ref src, ref dst, _allocator.ValueLength); } + public void UpsertCompletionCallback(ref Key key, ref Value value, Empty ctx) { } + public void DeleteCompletionCallback(ref Key key, Empty ctx) { } + } + + private sealed class LogCompactFunctions : IFunctions + where CompactionFunctions : ICompactionFunctions + { + private readonly CompactionFunctions _functions; + + public LogCompactFunctions(CompactionFunctions functions) { - var srcLength = allocator.ValueLength.GetLength(ref src); - var dstLength = allocator.ValueLength.GetLength(ref dst); + _functions = functions; + } + public void CheckpointCompletionCallback(string sessionId, CommitPoint commitPoint) { } + public void ConcurrentReader(ref Key key, ref Empty input, ref Value value, ref Empty dst) { } + public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) { return _functions.CopyInPlace(ref src, ref dst, null); } + public void CopyUpdater(ref Key key, ref Empty input, ref Value oldValue, ref Value newValue) { } + public void InitialUpdater(ref Key key, ref Empty input, ref Value value) { } + public bool InPlaceUpdater(ref Key key, ref Empty input, ref Value value) { return true; } + public void ReadCompletionCallback(ref Key key, ref Empty input, ref Empty output, Empty ctx, Status status) { } + public void RMWCompletionCallback(ref Key key, ref Empty input, Empty ctx, Status status) { } + public void SingleReader(ref Key key, ref Empty input, ref Value value, ref Empty dst) { } + public void SingleWriter(ref Key key, ref Value src, ref Value dst) { _functions.Copy(ref src, ref dst, null); } + public void UpsertCompletionCallback(ref Key key, ref Value value, Empty ctx) { } + public void DeleteCompletionCallback(ref Key key, Empty ctx) { } + } + + private unsafe struct DefaultVariableCompactionFunctions : ICompactionFunctions + { + public void Copy(ref Value src, ref Value dst, IVariableLengthStruct valueLength) + { + var srcLength = valueLength.GetLength(ref src); + Buffer.MemoryCopy( + Unsafe.AsPointer(ref src), + Unsafe.AsPointer(ref dst), + srcLength, + srcLength); + } + + public bool CopyInPlace(ref Value src, ref Value dst, IVariableLengthStruct valueLength) + { + var srcLength = valueLength.GetLength(ref src); + var dstLength = valueLength.GetLength(ref dst); if (srcLength != dstLength) return false; - allocator.ShallowCopy(ref src, ref dst); + Buffer.MemoryCopy( + Unsafe.AsPointer(ref src), + Unsafe.AsPointer(ref dst), + dstLength, + srcLength); + return true; } - public void CopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue) { } - public void InitialUpdater(ref Key key, ref Input input, ref Value value) { } - public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value) => false; - public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, Context ctx, Status status) { } - public void RMWCompletionCallback(ref Key key, ref Input input, Context ctx, Status status) { } - public void SingleReader(ref Key key, ref Input input, ref Value value, ref Output dst) { } - public void SingleWriter(ref Key key, ref Value src, ref Value dst) { allocator.ShallowCopy(ref src, ref dst); } - public void UpsertCompletionCallback(ref Key key, ref Value value, Context ctx) { } - public void DeleteCompletionCallback(ref Key key, Context ctx) { } + + public bool IsDeleted(in Key key, in Value value) + { + return false; + } } - private sealed class LogCompactFunctions : IFunctions + private struct DefaultCompactionFunctions : ICompactionFunctions { - public void CheckpointCompletionCallback(string sessionId, CommitPoint commitPoint) { } - public void ConcurrentReader(ref Key key, ref Input input, ref Value value, ref Output dst) { } - public bool ConcurrentWriter(ref Key key, ref Value src, ref Value dst) { dst = src; return true; } - public void CopyUpdater(ref Key key, ref Input input, ref Value oldValue, ref Value newValue) { } - public void InitialUpdater(ref Key key, ref Input input, ref Value value) { } - public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value) { return true; } - public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, Context ctx, Status status) { } - public void RMWCompletionCallback(ref Key key, ref Input input, Context ctx, Status status) { } - public void SingleReader(ref Key key, ref Input input, ref Value value, ref Output dst) { } - public void SingleWriter(ref Key key, ref Value src, ref Value dst) { dst = src; } - public void UpsertCompletionCallback(ref Key key, ref Value value, Context ctx) { } - public void DeleteCompletionCallback(ref Key key, Context ctx) { } + public void Copy(ref Value src, ref Value dst, IVariableLengthStruct valueLength) + { + dst = src; + } + + public bool CopyInPlace(ref Value src, ref Value dst, IVariableLengthStruct valueLength) + { + dst = src; + return true; + } + + public bool IsDeleted(in Key key, in Value value) + { + return false; + } } } } diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 832a73b75..5b84b7370 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -241,27 +241,28 @@ public ValueTask EnqueueAsync(byte[] entry, CancellationToken token = defa return new ValueTask(logicalAddress); return SlowEnqueueAsync(this, entry, token); - static async ValueTask SlowEnqueueAsync(FasterLog @this, byte[] entry, CancellationToken token) + } + + private static async ValueTask SlowEnqueueAsync(FasterLog @this, byte[] entry, CancellationToken token) + { + long logicalAddress; + while (true) { - long logicalAddress; - while (true) + var task = @this.CommitTask; + if (@this.TryEnqueue(entry, out logicalAddress)) + break; + if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) { - var task = @this.CommitTask; - if (@this.TryEnqueue(entry, out logicalAddress)) - break; - if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) + // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) + try { - // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) - try - { - await task.WithCancellationAsync(token); - } - catch when (!token.IsCancellationRequested) { } + await task.WithCancellationAsync(token); } + catch when (!token.IsCancellationRequested) { } } - - return logicalAddress; } + + return logicalAddress; } /// @@ -278,27 +279,28 @@ public ValueTask EnqueueAsync(ReadOnlyMemory entry, CancellationToke return new ValueTask(logicalAddress); return SlowEnqueueAsync(this, entry, token); - static async ValueTask SlowEnqueueAsync(FasterLog @this, ReadOnlyMemory entry, CancellationToken token) + } + + private static async ValueTask SlowEnqueueAsync(FasterLog @this, ReadOnlyMemory entry, CancellationToken token) + { + long logicalAddress; + while (true) { - long logicalAddress; - while (true) + var task = @this.CommitTask; + if (@this.TryEnqueue(entry.Span, out logicalAddress)) + break; + if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) { - var task = @this.CommitTask; - if (@this.TryEnqueue(entry.Span, out logicalAddress)) - break; - if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) + // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) + try { - // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) - try - { - await task.WithCancellationAsync(token); - } - catch when (!token.IsCancellationRequested) { } + await task.WithCancellationAsync(token); } + catch when (!token.IsCancellationRequested) { } } - - return logicalAddress; } + + return logicalAddress; } /// @@ -315,27 +317,28 @@ public ValueTask EnqueueAsync(IReadOnlySpanBatch readOnlySpanBatch, Cancel return new ValueTask(address); return SlowEnqueueAsync(this, readOnlySpanBatch, token); - static async ValueTask SlowEnqueueAsync(FasterLog @this, IReadOnlySpanBatch readOnlySpanBatch, CancellationToken token) + } + + private static async ValueTask SlowEnqueueAsync(FasterLog @this, IReadOnlySpanBatch readOnlySpanBatch, CancellationToken token) + { + long logicalAddress; + while (true) { - long logicalAddress; - while (true) + var task = @this.CommitTask; + if (@this.TryEnqueue(readOnlySpanBatch, out logicalAddress)) + break; + if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) { - var task = @this.CommitTask; - if (@this.TryEnqueue(readOnlySpanBatch, out logicalAddress)) - break; - if (@this.NeedToWait(@this.CommittedUntilAddress, @this.TailAddress)) + // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) + try { - // Wait for *some* commit - failure can be ignored except if the token was signaled (which the caller should handle correctly) - try - { - await task.WithCancellationAsync(token); - } - catch when (!token.IsCancellationRequested) { } + await task.WithCancellationAsync(token); } + catch when (!token.IsCancellationRequested) { } } - - return logicalAddress; } + + return logicalAddress; } #endregion diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 4dbbe074f..45c8ec566 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -153,25 +153,24 @@ public ValueTask WaitAsync(CancellationToken token = default) return new ValueTask(true); return SlowWaitAsync(this, token); + } - // use static local function to guarantee there's no accidental closure getting allocated here - static async ValueTask SlowWaitAsync(FasterLogScanIterator @this, CancellationToken token) + private static async ValueTask SlowWaitAsync(FasterLogScanIterator @this, CancellationToken token) + { + while (true) { - while (true) + if (@this.disposed) + return false; + var commitTask = @this.fasterLog.CommitTask; + if (@this.NextAddress < @this.fasterLog.CommittedUntilAddress) + return true; + // Ignore commit exceptions, except when the token is signaled + try { - if (@this.disposed) - return false; - var commitTask = @this.fasterLog.CommitTask; - if (@this.NextAddress < @this.fasterLog.CommittedUntilAddress) - return true; - // Ignore commit exceptions, except when the token is signaled - try - { - await commitTask.WithCancellationAsync(token); - } - catch (ObjectDisposedException) { return false; } - catch when (!token.IsCancellationRequested) { } + await commitTask.WithCancellationAsync(token); } + catch (ObjectDisposedException) { return false; } + catch when (!token.IsCancellationRequested) { } } } diff --git a/cs/src/core/Index/Interfaces/ICompactionFunctions.cs b/cs/src/core/Index/Interfaces/ICompactionFunctions.cs new file mode 100644 index 000000000..518c951e2 --- /dev/null +++ b/cs/src/core/Index/Interfaces/ICompactionFunctions.cs @@ -0,0 +1,64 @@ +namespace FASTER.core +{ + /// + /// Optional functions to be called during compaction. + /// + /// + /// + public interface ICompactionFunctions + { + /// + /// Checks if record in the faster log is logically deleted. + /// If the record was deleted via + /// or + /// then this function is not called for such a record. + /// + /// + /// + /// One possible scenario is if FASTER is used to store reference counted records. + /// Once the record count reaches zero it can be considered to be no longer relevant and + /// compaction can skip the record. + /// + /// + /// Compaction might be implemented by scanning the log thus it is possible that multiple + /// records with the same key and/or different value might be provided. Only the last record will be persisted. + /// + /// + /// This method can be called concurrently with methods in . It is responsibility + /// of the implementer to correctly manage concurrency. + /// + /// + /// + /// + /// + bool IsDeleted(in Key key, in Value value); + + /// + /// Copies a value from to . + /// It is possible that value at might be modified during copy operation thus to prevent torn writes + /// this method is provided to allow the implementer to correctly handle concurrency. + /// This method is counterpart to . + /// + /// Managed pointer to value at source + /// Managed pointer to uninitialized value at destination + /// [Can be null] Variable length struct functions + /// + void Copy(ref Value src, ref Value dst, IVariableLengthStruct valueLength); + + /// + /// Copies a value from to . + /// It is possible that value at might be modified during copy operation thus to prevent torn writes + /// this method is provided to allow the implementer to correctly handle concurrency. + /// This method is counterpart to . + /// + /// Managed pointer to value at source + /// Managed pointer to existing value at destination + /// [Can be null] Variable length struct functions + /// + /// True - if can be safely copied to (see ). + /// False - if a new record needs to be allocated. In this case will be called with + /// managed pointer to new record. + /// + bool CopyInPlace(ref Value src, ref Value dst, IVariableLengthStruct valueLength); + } +} \ No newline at end of file diff --git a/cs/src/core/Index/Interfaces/IFasterKV.cs b/cs/src/core/Index/Interfaces/IFasterKV.cs index 2abeaebfa..15b02dd63 100644 --- a/cs/src/core/Index/Interfaces/IFasterKV.cs +++ b/cs/src/core/Index/Interfaces/IFasterKV.cs @@ -212,12 +212,137 @@ public interface IFasterKV : IDis /// /// Get accessor for FASTER hybrid log /// - LogAccessor Log { get; } + LogAccessor Log { get; } /// /// Get accessor for FASTER read cache /// - LogAccessor ReadCache { get; } + LogAccessor ReadCache { get; } + + #endregion + } + + /// + /// Interface to FASTER key-value store + /// + public interface IFasterKV : IDisposable + where Key : new() + where Value : new() + { + #region New Session Operations + + /// + /// Start a new client session with FASTER. + /// + /// Callback functions. + /// ID/name of session (auto-generated if not provided) + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + ClientSession NewSession(Functions functions, string sessionId = null, bool threadAffinitized = false) + where Functions : IFunctions; + + /// + /// Resume (continue) prior client session with FASTER, used during + /// recovery from failure. + /// + /// Callback functions. + /// ID/name of previous session to resume + /// Prior commit point of durability for session + /// For advanced users. Specifies whether session holds the thread epoch across calls. Do not use with async code. Ensure thread calls session Refresh periodically to move the system epoch forward. + /// Session instance + ClientSession ResumeSession(Functions functions, string sessionId, out CommitPoint commitPoint, bool threadAffinitized = false) + where Functions : IFunctions; + + #endregion + + #region Growth and Recovery + + /// + /// Grow the hash index + /// + /// + bool GrowIndex(); + + /// + /// Take full checkpoint of FASTER + /// + /// Token describing checkpoint + /// upper limit (inclusive) of the version included + /// Whether checkpoint was initiated + bool TakeFullCheckpoint(out Guid token, long targetVersion = -1); + + /// + /// Take checkpoint of FASTER index only (not log) + /// + /// Token describing checkpoin + /// Whether checkpoint was initiated + bool TakeIndexCheckpoint(out Guid token); + + /// + /// Take checkpoint of FASTER log only (not index) + /// + /// Token describing checkpoin + /// upper limit (inclusive) of the version included + /// Whether checkpoint was initiated + bool TakeHybridLogCheckpoint(out Guid token, long targetVersion = -1); + + /// + /// Recover from last successfuly checkpoints + /// + void Recover(); + + /// + /// Recover using full checkpoint token + /// + /// + void Recover(Guid fullcheckpointToken); + + /// + /// Recover using a separate index and log checkpoint token + /// + /// + /// + void Recover(Guid indexToken, Guid hybridLogToken); + + /// + /// Complete ongoing checkpoint (spin-wait) + /// + /// Whether checkpoint has completed + ValueTask CompleteCheckpointAsync(CancellationToken token = default); + + #endregion + + #region Other Operations + + /// + /// Get number of (non-zero) hash entries in FASTER + /// + long EntryCount { get; } + + /// + /// Get size of index in #cache lines (64 bytes each) + /// + long IndexSize { get; } + + /// + /// Get comparer used by this instance of FASTER + /// + IFasterEqualityComparer Comparer { get; } + + /// + /// Dump distribution of #entries in hash table + /// + string DumpDistribution(); + + /// + /// Get accessor for FASTER hybrid log + /// + LogAccessor Log { get; } + + /// + /// Get accessor for FASTER read cache + /// + LogAccessor ReadCache { get; } #endregion } diff --git a/cs/src/core/Index/Interfaces/IFasterSession.cs b/cs/src/core/Index/Interfaces/IFasterSession.cs new file mode 100644 index 000000000..d03d265d6 --- /dev/null +++ b/cs/src/core/Index/Interfaces/IFasterSession.cs @@ -0,0 +1,25 @@ +namespace FASTER.core +{ + /// + /// Provides thread management and callback to checkpoint completion (called state machine). + /// + // This is split to two interfaces just to limit infection of type parameters + internal interface IFasterSession + { + void UnsafeResumeThread(); + void UnsafeSuspendThread(); + void CheckpointCompletionCallback(string sessionId, CommitPoint commitPoint); + } + + /// + /// Provides thread management and all callbacks. + /// + /// + /// + /// + /// + /// + internal interface IFasterSession : IFunctions, IFasterSession + { + } +} \ No newline at end of file diff --git a/cs/src/core/Index/Interfaces/NullFasterSession.cs b/cs/src/core/Index/Interfaces/NullFasterSession.cs new file mode 100644 index 000000000..3c340474d --- /dev/null +++ b/cs/src/core/Index/Interfaces/NullFasterSession.cs @@ -0,0 +1,19 @@ +namespace FASTER.core +{ + struct NullFasterSession : IFasterSession + { + public static readonly NullFasterSession Instance; + + public void CheckpointCompletionCallback(string guid, CommitPoint commitPoint) + { + } + + public void UnsafeResumeThread() + { + } + + public void UnsafeSuspendThread() + { + } + } +} \ No newline at end of file diff --git a/cs/src/core/Index/Recovery/Checkpoint.cs b/cs/src/core/Index/Recovery/Checkpoint.cs index c06a178bd..c9af0656a 100644 --- a/cs/src/core/Index/Recovery/Checkpoint.cs +++ b/cs/src/core/Index/Recovery/Checkpoint.cs @@ -32,10 +32,9 @@ internal static class EpochPhaseIdx public const int CheckpointCompletionCallback = 5; } - public partial class FasterKV + public partial class FasterKV where Key : new() where Value : new() - where Functions : IFunctions { internal TaskCompletionSource checkpointTcs @@ -47,7 +46,7 @@ internal TaskCompletionSource checkpointTcs internal Task CheckpointTask => checkpointTcs.Task; - internal void AcquireSharedLatchesForAllPendingRequests(FasterExecutionContext ctx) + internal void AcquireSharedLatchesForAllPendingRequests(FasterExecutionContext ctx) { foreach (var _ctx in ctx.retryRequests) { diff --git a/cs/src/core/Index/Recovery/Recovery.cs b/cs/src/core/Index/Recovery/Recovery.cs index fc66af57d..51aac3b73 100644 --- a/cs/src/core/Index/Recovery/Recovery.cs +++ b/cs/src/core/Index/Recovery/Recovery.cs @@ -45,10 +45,9 @@ public RecoveryStatus(int capacity, } - public unsafe partial class FasterKV : FasterBase, IFasterKV + public unsafe partial class FasterKV : FasterBase, IFasterKV where Key : new() where Value : new() - where Functions : IFunctions { private void InternalRecoverFromLatestCheckpoints() @@ -90,8 +89,8 @@ private void InternalRecover(Guid indexToken, Guid hybridLogToken) // Set new system state after recovery var v = recoveredHLCInfo.info.version; - _systemState.phase = Phase.REST; - _systemState.version = (v + 1); + systemState.phase = Phase.REST; + systemState.version = (v + 1); // Recover fuzzy index from checkpoint RecoverFuzzyIndex(recoveredICInfo); diff --git a/cs/src/core/Index/Synchronization/FasterStateMachine.cs b/cs/src/core/Index/Synchronization/FasterStateMachine.cs index 3edf21a72..dfadbc086 100644 --- a/cs/src/core/Index/Synchronization/FasterStateMachine.cs +++ b/cs/src/core/Index/Synchronization/FasterStateMachine.cs @@ -6,15 +6,14 @@ namespace FASTER.core { - public partial class FasterKV + public partial class FasterKV where Key : new() where Value : new() - where Functions : IFunctions { // 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 - private SystemState _systemState; + 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 @@ -34,7 +33,7 @@ private bool StartStateMachine(ISynchronizationStateMachine stateMachine) currentSyncStateMachine = stateMachine; // No latch required because the taskMutex guards against other tasks starting, and only a new task // is allowed to change faster global state from REST - GlobalStateMachineStep(_systemState); + GlobalStateMachineStep(systemState); return true; } @@ -42,7 +41,7 @@ private bool StartStateMachine(ISynchronizationStateMachine stateMachine) [MethodImpl(MethodImplOptions.AggressiveInlining)] private bool MakeTransition(SystemState expectedState, SystemState nextState) { - if (Interlocked.CompareExchange(ref _systemState.word, nextState.word, expectedState.word) != + if (Interlocked.CompareExchange(ref systemState.word, nextState.word, expectedState.word) != expectedState.word) return false; Debug.WriteLine("Moved to {0}, {1}", nextState.phase, nextState.version); return true; @@ -107,7 +106,7 @@ private SystemState FastForwardToCurrentCycle(SystemState currentThreadState, Sy while (true) { var task = currentSyncStateMachine; - var targetState = SystemState.Copy(ref _systemState); + var targetState = SystemState.Copy(ref systemState); // We have to make sure that we are not looking at a state resulted from a different // task. It's ok to be behind when the thread steps through the state machine, but not // ok if we are using the wrong task. @@ -121,17 +120,19 @@ private SystemState FastForwardToCurrentCycle(SystemState currentThreadState, Sy /// 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) - /// null if calling without a session (e.g. waiting on a checkpoint) + /// Faster session. /// /// /// - private async ValueTask ThreadStateMachineStep(FasterExecutionContext ctx, - ClientSession clientSession, + private async ValueTask ThreadStateMachineStep( + FasterExecutionContext ctx, + FasterSession fasterSession, bool async = true, CancellationToken token = default) + where FasterSession : IFasterSession { if (async) - clientSession?.UnsafeResumeThread(); + fasterSession.UnsafeResumeThread(); // Target state is the current (non-intermediate state) system state thread needs to catch up to var (currentTask, targetState) = CaptureTaskAndTargetState(); @@ -159,8 +160,7 @@ private async ValueTask ThreadStateMachineStep(FasterExecutionContext ctx, var previousState = threadState; do { - await currentTask.OnThreadEnteringState(threadState, previousState, this, ctx, clientSession, async, - token); + await currentTask.OnThreadEnteringState(threadState, previousState, this, ctx, fasterSession, async, token); if (ctx != null) { ctx.phase = threadState.phase; diff --git a/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs b/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs index 90b6f0bc4..00d10473b 100644 --- a/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs +++ b/cs/src/core/Index/Synchronization/FullCheckpointStateMachine.cs @@ -11,12 +11,11 @@ namespace FASTER.core internal sealed class FullCheckpointOrchestrationTask : ISynchronizationTask { /// - public void GlobalBeforeEnteringState( + public void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { switch (next.phase) { @@ -46,38 +45,39 @@ public void GlobalBeforeEnteringState - public void GlobalAfterEnteringState( + public void GlobalAfterEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } /// - public async ValueTask OnThreadState(SystemState current, + public async ValueTask OnThreadState( + SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { if (current.phase != Phase.WAIT_INDEX_CHECKPOINT) return; if (async && !faster.IsIndexFuzzyCheckpointCompleted()) { - clientSession?.UnsafeSuspendThread(); + fasterSession?.UnsafeSuspendThread(); await faster.IsIndexFuzzyCheckpointCompletedAsync(token); - clientSession?.UnsafeResumeThread(); + fasterSession?.UnsafeResumeThread(); } faster.GlobalStateMachineStep(current); } } - + /// /// The state machine orchestrates a full checkpoint /// @@ -91,7 +91,8 @@ internal sealed class FullCheckpointStateMachine : HybridLogCheckpointStateMachi /// upper limit (inclusive) of the version included public FullCheckpointStateMachine(ISynchronizationTask checkpointBackend, long targetVersion = -1) : base( targetVersion, new VersionChangeTask(), new FullCheckpointOrchestrationTask(), checkpointBackend, - new IndexSnapshotTask()) {} + new IndexSnapshotTask()) + { } /// public override SystemState NextState(SystemState start) diff --git a/cs/src/core/Index/Synchronization/HybridLogCheckpointTask.cs b/cs/src/core/Index/Synchronization/HybridLogCheckpointTask.cs index c73825999..ced26a97a 100644 --- a/cs/src/core/Index/Synchronization/HybridLogCheckpointTask.cs +++ b/cs/src/core/Index/Synchronization/HybridLogCheckpointTask.cs @@ -12,11 +12,10 @@ namespace FASTER.core internal abstract class HybridLogCheckpointOrchestrationTask : ISynchronizationTask { /// - public virtual void GlobalBeforeEnteringState(SystemState next, - FasterKV faster) + public virtual void GlobalBeforeEnteringState(SystemState next, + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { switch (next.phase) { @@ -48,10 +47,10 @@ public virtual void GlobalBeforeEnteringState - public virtual void GlobalAfterEnteringState(SystemState next, - FasterKV faster) + public virtual void GlobalAfterEnteringState(SystemState next, + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } /// - public virtual ValueTask OnThreadState( + public virtual ValueTask OnThreadState( SystemState current, - SystemState prev, FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + SystemState prev, FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { if (current.phase != Phase.PERSISTENCE_CALLBACK) return default; @@ -99,8 +98,7 @@ public virtual ValueTask OnThreadState - public override void GlobalBeforeEnteringState(SystemState next, - FasterKV faster) + public override void GlobalBeforeEnteringState(SystemState next, + FasterKV faster) { base.GlobalBeforeEnteringState(next, faster); if (next.phase != Phase.WAIT_FLUSH) return; @@ -135,14 +133,16 @@ public override void GlobalBeforeEnteringState - public override async ValueTask OnThreadState( + public override async ValueTask OnThreadState( SystemState current, - SystemState prev, FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + SystemState prev, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) { - await base.OnThreadState(current, prev, faster, ctx, clientSession, async, token); + await base.OnThreadState(current, prev, faster, ctx, fasterSession, async, token); if (current.phase != Phase.WAIT_FLUSH) return; if (ctx == null || !ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush]) @@ -153,9 +153,9 @@ public override async ValueTask OnThreadState - public override void GlobalBeforeEnteringState(SystemState next, - FasterKV faster) + public override void GlobalBeforeEnteringState(SystemState next, FasterKV faster) { base.GlobalBeforeEnteringState(next, faster); switch (next.phase) @@ -224,14 +223,15 @@ public override void GlobalBeforeEnteringState - public override async ValueTask OnThreadState( + public override async ValueTask OnThreadState( SystemState current, - SystemState prev, FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + SystemState prev, FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) { - await base.OnThreadState(current, prev, faster, ctx, clientSession, async, token); + await base.OnThreadState(current, prev, faster, ctx, fasterSession, async, token); if (current.phase != Phase.WAIT_FLUSH) return; if (ctx == null || !ctx.prevCtx.markers[EpochPhaseIdx.WaitFlush]) @@ -242,15 +242,15 @@ public override async ValueTask OnThreadStateA 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) {} + : base(targetVersion, new VersionChangeTask(), checkpointBackend) { } /// /// Construct a new HybridLogCheckpointStateMachine with the given tasks. Does not load any tasks by default. @@ -283,7 +283,7 @@ public HybridLogCheckpointStateMachine(ISynchronizationTask checkpointBackend, l /// 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) {} + : base(targetVersion, tasks) { } /// public override SystemState NextState(SystemState start) diff --git a/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs b/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs index 0f2bcff6f..a59f402f7 100644 --- a/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs +++ b/cs/src/core/Index/Synchronization/ISynchronizationStateMachine.cs @@ -1,4 +1,5 @@ -using System.Globalization; +using System; +using System.Globalization; using System.Threading; using System.Threading.Tasks; @@ -25,15 +26,10 @@ internal interface ISynchronizationStateMachine /// /// /// - /// - /// - /// - /// - void GlobalBeforeEnteringState(SystemState next, - FasterKV faster) + void GlobalBeforeEnteringState(SystemState next, + FasterKV faster) where Key : new() - where Value : new() - where Functions : IFunctions; + where Value : new(); /// /// This function is invoked immediately after the global state machine enters the given state. @@ -42,15 +38,10 @@ void GlobalBeforeEnteringState(Sy /// /// /// - /// - /// - /// - /// - void GlobalAfterEnteringState(SystemState next, - FasterKV faster) + void GlobalAfterEnteringState(SystemState next, + FasterKV faster) where Key : new() - where Value : new() - where Functions : IFunctions; + where Value : new(); /// /// This function is invoked for every thread when they refresh and observe a given state. @@ -61,7 +52,7 @@ void GlobalAfterEnteringState(Sys /// /// /// - /// + /// /// /// /// @@ -69,21 +60,20 @@ void GlobalAfterEnteringState(Sys /// /// /// - /// + /// /// - ValueTask OnThreadEnteringState(SystemState current, + ValueTask OnThreadEnteringState(SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions; + where FasterSession: IFasterSession; } - /// /// 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 @@ -99,16 +89,11 @@ internal interface ISynchronizationTask /// /// /// - /// - /// - /// - /// - void GlobalBeforeEnteringState( + void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() - where Value : new() - where Functions : IFunctions; + where Value : new(); /// /// This function is invoked immediately after the global state machine enters the given state. @@ -117,16 +102,11 @@ void GlobalBeforeEnteringState( /// /// /// - /// - /// - /// - /// - void GlobalAfterEnteringState( + void GlobalAfterEnteringState( SystemState next, - FasterKV faster) - where Key : new () - where Value : new () - where Functions : IFunctions; + FasterKV faster) + where Key : new() + where Value : new(); /// /// This function is invoked for every thread when they refresh and observe a given state. @@ -137,7 +117,7 @@ void GlobalAfterEnteringState( /// /// /// - /// + /// /// /// /// @@ -145,22 +125,21 @@ void GlobalAfterEnteringState( /// /// /// - /// + /// /// - ValueTask OnThreadState( + ValueTask OnThreadState( SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions; + where FasterSession: IFasterSession; } - /// /// Abstract base class for ISynchronizationStateMachine that implements that state machine logic /// with ISynchronizationTasks @@ -183,38 +162,37 @@ protected SynchronizationStateMachineBase(params ISynchronizationTask[] tasks) public abstract SystemState NextState(SystemState start); /// - public void GlobalBeforeEnteringState(SystemState next, - FasterKV faster) where Key : new() + public void GlobalBeforeEnteringState(SystemState next, + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { foreach (var task in tasks) task.GlobalBeforeEnteringState(next, faster); } /// - public void GlobalAfterEnteringState(SystemState next, - FasterKV faster) where Key : new() + public void GlobalAfterEnteringState(SystemState next, + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { foreach (var task in tasks) task.GlobalAfterEnteringState(next, faster); } /// - public async ValueTask OnThreadEnteringState( + public async ValueTask OnThreadEnteringState( SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession: IFasterSession { foreach (var task in tasks) - await task.OnThreadState(current, prev, faster, ctx, clientSession, async, token); + await task.OnThreadState(current, prev, faster, ctx, fasterSession, async, token); } } } \ No newline at end of file diff --git a/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs b/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs index fc37bad01..39180539a 100644 --- a/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs +++ b/cs/src/core/Index/Synchronization/IndexResizeStateMachine.cs @@ -9,12 +9,11 @@ namespace FASTER.core internal sealed class IndexResizeTask : ISynchronizationTask { /// - public void GlobalBeforeEnteringState( + public void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { switch (next.phase) { @@ -29,8 +28,7 @@ public void GlobalBeforeEnteringState - public void GlobalAfterEnteringState( + public void GlobalAfterEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { switch (next.phase) { @@ -65,17 +62,17 @@ public void GlobalAfterEnteringState - public ValueTask OnThreadState( + public ValueTask OnThreadState( SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { switch (current.phase) { diff --git a/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs b/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs index 65e7f2ae8..2f71446a9 100644 --- a/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs +++ b/cs/src/core/Index/Synchronization/IndexSnapshotStateMachine.cs @@ -11,12 +11,11 @@ namespace FASTER.core internal class IndexSnapshotTask : ISynchronizationTask { /// - public void GlobalBeforeEnteringState( + public void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { switch (next.phase) { @@ -53,25 +52,26 @@ public void GlobalBeforeEnteringState - public void GlobalAfterEnteringState( + public void GlobalAfterEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } /// - public async ValueTask OnThreadState( + public async ValueTask OnThreadState( SystemState current, - SystemState prev, FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + SystemState prev, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { switch (current.phase) { @@ -95,9 +95,9 @@ public async ValueTask OnThreadState - public void GlobalBeforeEnteringState( + public void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } /// - public void GlobalAfterEnteringState( + public void GlobalAfterEnteringState( SystemState start, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } /// - public ValueTask OnThreadState( + public ValueTask OnThreadState( SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { switch (current.phase) { @@ -85,7 +83,7 @@ public ValueTask OnThreadState( case Phase.WAIT_PENDING: if (ctx != null) { - if (!faster.RelaxedCPR &&!ctx.prevCtx.markers[EpochPhaseIdx.WaitPending]) + if (!faster.RelaxedCPR && !ctx.prevCtx.markers[EpochPhaseIdx.WaitPending]) { if (ctx.prevCtx.HasNoPendingRequests) ctx.prevCtx.markers[EpochPhaseIdx.WaitPending] = true; @@ -114,12 +112,11 @@ public ValueTask OnThreadState( internal sealed class FoldOverTask : ISynchronizationTask { /// - public void GlobalBeforeEnteringState( + public void GlobalBeforeEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { if (next.phase == Phase.REST) // Before leaving the checkpoint, make sure all previous versions are read-only. @@ -127,24 +124,25 @@ public void GlobalBeforeEnteringState - public void GlobalAfterEnteringState( + public void GlobalAfterEnteringState( SystemState next, - FasterKV faster) + FasterKV faster) where Key : new() where Value : new() - where Functions : IFunctions { } + { } /// - public ValueTask OnThreadState( + public ValueTask OnThreadState( SystemState current, SystemState prev, - FasterKV faster, - FasterKV.FasterExecutionContext ctx, - ClientSession clientSession, bool async = true, + FasterKV faster, + FasterKV.FasterExecutionContext ctx, + FasterSession fasterSession, + bool async = true, CancellationToken token = default) where Key : new() where Value : new() - where Functions : IFunctions + where FasterSession : IFasterSession { return default; } @@ -156,7 +154,7 @@ public ValueTask OnThreadState( internal class VersionChangeStateMachine : SynchronizationStateMachineBase { private readonly long targetVersion; - + /// /// Construct a new VersionChangeStateMachine with the given tasks. Does not load any tasks by default. /// @@ -185,7 +183,7 @@ public override SystemState NextState(SystemState start) case Phase.PREPARE: nextState.phase = Phase.IN_PROGRESS; // TODO: Move to long for system state as well. - nextState.version = (int) (targetVersion == -1 ? start.version + 1 : targetVersion + 1); + nextState.version = (int)(targetVersion == -1 ? start.version + 1 : targetVersion + 1); break; case Phase.IN_PROGRESS: // This phase has no effect if using relaxed CPR model diff --git a/cs/src/core/Utilities/AsyncResultTypes.cs b/cs/src/core/Utilities/AsyncResultTypes.cs index 7ea589ce0..65617b038 100644 --- a/cs/src/core/Utilities/AsyncResultTypes.cs +++ b/cs/src/core/Utilities/AsyncResultTypes.cs @@ -69,18 +69,4 @@ internal struct OverflowPagesReadAsyncResult : IAsyncResult public bool CompletedSynchronously => throw new NotImplementedException(); } - - internal struct CountdownEventAsyncResult : IAsyncResult - { - public CountdownEvent countdown; - public Action action; - - public bool IsCompleted => throw new NotImplementedException(); - - public WaitHandle AsyncWaitHandle => throw new NotImplementedException(); - - public object AsyncState => throw new NotImplementedException(); - - public bool CompletedSynchronously => throw new NotImplementedException(); - } } diff --git a/cs/src/core/Utilities/PageAsyncResultTypes.cs b/cs/src/core/Utilities/PageAsyncResultTypes.cs index 9c63d2573..b126d2215 100644 --- a/cs/src/core/Utilities/PageAsyncResultTypes.cs +++ b/cs/src/core/Utilities/PageAsyncResultTypes.cs @@ -91,7 +91,6 @@ public sealed class PageAsyncFlushResult : IAsyncResult internal bool partial; internal long fromAddress; internal long untilAddress; - internal IDevice objlogDevice; internal SectorAlignedMemory freeBuffer1; internal SectorAlignedMemory freeBuffer2; internal AutoResetEvent done; diff --git a/cs/src/core/Utilities/Utility.cs b/cs/src/core/Utilities/Utility.cs index e1fbe4ce8..9f690362a 100644 --- a/cs/src/core/Utilities/Utility.cs +++ b/cs/src/core/Utilities/Utility.cs @@ -300,22 +300,22 @@ internal static Task WithCancellationAsync(this Task task, Cancellation return Task.FromCanceled(token); } - return Inner(task, token, useSynchronizationContext, continueOnCapturedContext); + return SlowWithCancellationAsync(task, token, useSynchronizationContext, continueOnCapturedContext); + } - static async Task Inner(Task task, CancellationToken token, bool useSynchronizationContext, bool continueOnCapturedContext) + private static async Task SlowWithCancellationAsync(Task task, CancellationToken token, bool useSynchronizationContext, bool continueOnCapturedContext) + { + var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + using (token.Register(s => ((TaskCompletionSource)s).TrySetResult(true), tcs, useSynchronizationContext)) { - var tcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - using (token.Register(s => ((TaskCompletionSource)s).TrySetResult(true), tcs, useSynchronizationContext)) + if (task != await Task.WhenAny(task, tcs.Task)) { - if (task != await Task.WhenAny(task, tcs.Task)) - { - token.ThrowIfCancellationRequested(); - } + token.ThrowIfCancellationRequested(); } - - // make sure any exceptions in the task get unwrapped and exposed to the caller. - return await task.ConfigureAwait(continueOnCapturedContext); } + + // make sure any exceptions in the task get unwrapped and exposed to the caller. + return await task.ConfigureAwait(continueOnCapturedContext); } } } \ No newline at end of file diff --git a/cs/test/BlittableLogCompactionTests.cs b/cs/test/BlittableLogCompactionTests.cs index 83c6c335e..beb471ff5 100644 --- a/cs/test/BlittableLogCompactionTests.cs +++ b/cs/test/BlittableLogCompactionTests.cs @@ -196,5 +196,141 @@ public void BlittableLogCompactionTest3() } } + [Test] + public void BlittableLogCompactionCustomFunctionsTest1() + { + using var session = fht.NewSession(); + + InputStruct input = default; + + const int totalRecords = 2000; + var start = fht.Log.TailAddress; + var compactUntil = 0L; + + for (var i = 0; i < totalRecords; i++) + { + if (i == totalRecords / 2) + compactUntil = fht.Log.TailAddress; + + var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + session.Upsert(ref key1, ref value, 0, 0); + } + + var tail = fht.Log.TailAddress; + + // Only leave records with even vfield1 + fht.Log.Compact(default(EvenCompactionFunctions), compactUntil); + Assert.IsTrue(fht.Log.BeginAddress == compactUntil); + + // Read 2000 keys - all should be present + for (var i = 0; i < totalRecords; i++) + { + OutputStruct output = default; + var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; + var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; + + var ctx = (i < (totalRecords / 2) && (i % 2 != 0)) ? 1 : 0; + + var status = session.Read(ref key1, ref input, ref output, ctx, 0); + if (status == Status.PENDING) + { + session.CompletePending(true); + } + else + { + if (ctx == 0) + { + Assert.IsTrue(status == Status.OK); + Assert.IsTrue(output.value.vfield1 == value.vfield1); + Assert.IsTrue(output.value.vfield2 == value.vfield2); + } + else + { + Assert.IsTrue(status == Status.NOTFOUND); + } + } + } + } + + [Test] + public void BlittableLogCompactionCustomFunctionsTest2() + { + // This test checks if CopyInPlace returning false triggers call to Copy + + using var session = fht.NewSession(); + + var key = new KeyStruct { kfield1 = 100, kfield2 = 101 }; + var value = new ValueStruct { vfield1 = 10, vfield2 = 20 }; + + session.Upsert(ref key, ref value, 0, 0); + + fht.Log.Flush(true); + + value = new ValueStruct { vfield1 = 11, vfield2 = 21 }; + session.Upsert(ref key, ref value, 0, 0); + + fht.Log.Flush(true); + + var compactionFunctions = new Test2CompactionFunctions(); + fht.Log.Compact(compactionFunctions, fht.Log.TailAddress); + + Assert.IsTrue(compactionFunctions.CopyCalled); + + var input = default(InputStruct); + var output = default(OutputStruct); + var status = session.Read(ref key, ref input, ref output, 0, 0); + if (status == Status.PENDING) + { + session.CompletePending(true); + } + else + { + Assert.IsTrue(status == Status.OK); + Assert.IsTrue(output.value.vfield1 == value.vfield1); + Assert.IsTrue(output.value.vfield2 == value.vfield2); + } + } + + private class Test2CompactionFunctions : ICompactionFunctions + { + public bool CopyCalled; + + public void Copy(ref ValueStruct src, ref ValueStruct dst, IVariableLengthStruct valueLength) + { + if (src.vfield1 == 11 && src.vfield2 == 21) + CopyCalled = true; + dst = src; + } + + public bool CopyInPlace(ref ValueStruct src, ref ValueStruct dst, IVariableLengthStruct valueLength) + { + return false; + } + + public bool IsDeleted(in KeyStruct key, in ValueStruct value) + { + return false; + } + } + + private struct EvenCompactionFunctions : ICompactionFunctions + { + public void Copy(ref ValueStruct src, ref ValueStruct dst, IVariableLengthStruct valueLength) + { + dst = src; + } + + public bool CopyInPlace(ref ValueStruct src, ref ValueStruct dst, IVariableLengthStruct valueLength) + { + dst = src; + return true; + } + + public bool IsDeleted(in KeyStruct key, in ValueStruct value) + { + return value.vfield1 % 2 != 0; + } + } } } diff --git a/cs/test/FunctionPerSessionTests.cs b/cs/test/FunctionPerSessionTests.cs new file mode 100644 index 000000000..d8e2485a6 --- /dev/null +++ b/cs/test/FunctionPerSessionTests.cs @@ -0,0 +1,172 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; +using FASTER.core; +using NUnit.Framework; + +namespace FASTER.test +{ + public struct RefCountedValue + { + public int ReferenceCount; + public long Value; + } + + public class RefCountedAdder : FunctionsBase + { + public int InitialCount; + public int InPlaceCount; + public int CopyCount; + + public override void InitialUpdater(ref int key, ref long input, ref RefCountedValue value) + { + Interlocked.Increment(ref InitialCount); + + value.Value = input; + value.ReferenceCount = 1; + } + + public override bool InPlaceUpdater(ref int key, ref long input, ref RefCountedValue value) + { + Interlocked.Increment(ref InPlaceCount); + + value.Value = input; + value.ReferenceCount++; + return true; + } + + public override void CopyUpdater(ref int key, ref long input, ref RefCountedValue oldValue, ref RefCountedValue newValue) + { + Interlocked.Increment(ref CopyCount); + + newValue.Value = input; + newValue.ReferenceCount = oldValue.ReferenceCount + 1; + } + } + + public class RefCountedRemover : FunctionsBase + { + public int InitialCount; + public int InPlaceCount; + public int CopyCount; + + public override void InitialUpdater(ref int key, ref Empty input, ref RefCountedValue value) + { + Interlocked.Increment(ref InitialCount); + + value.Value = 0; + value.ReferenceCount = 0; + } + + public override bool InPlaceUpdater(ref int key, ref Empty input, ref RefCountedValue value) + { + Interlocked.Increment(ref InPlaceCount); + + if (value.ReferenceCount > 0) + value.ReferenceCount--; + + return true; + } + + public override void CopyUpdater(ref int key, ref Empty input, ref RefCountedValue oldValue, ref RefCountedValue newValue) + { + Interlocked.Increment(ref CopyCount); + + newValue.ReferenceCount = oldValue.ReferenceCount; + if (newValue.ReferenceCount > 0) + newValue.ReferenceCount--; + newValue.Value = oldValue.Value; + } + } + + public class RefCountedReader : FunctionsBase + { + public override void SingleReader(ref int key, ref Empty input, ref RefCountedValue value, ref RefCountedValue dst) + { + dst = value; + } + + public override void ConcurrentReader(ref int key, ref Empty input, ref RefCountedValue value, ref RefCountedValue dst) + { + dst = value; + } + } + + [TestFixture] + public class FunctionPerSessionTests + { + private IDevice _log; + private FasterKV _faster; + private RefCountedAdder _adder; + private RefCountedRemover _remover; + private RefCountedReader _reader; + + [SetUp] + public void Setup() + { + _log = Devices.CreateLogDevice(TestContext.CurrentContext.TestDirectory + "\\FunctionPerSessionTests1.log", deleteOnClose: true); + + _faster = new FasterKV(128, new LogSettings() + { + LogDevice = _log, + }); + + _adder = new RefCountedAdder(); + _remover = new RefCountedRemover(); + _reader = new RefCountedReader(); + } + + [TearDown] + public void TearDown() + { + _faster.Dispose(); + _faster = null; + _log.Close(); + } + + [Test] + public async Task Should_create_multiple_sessions_with_different_callbacks() + { + using (var adderSession = _faster.NewSession(_adder)) + using (var removerSession = _faster.NewSession(_remover)) + using (var readerSession = _faster.NewSession(_reader)) + { + var key = 101; + var input = 1000L; + + await adderSession.RMWAsync(ref key, ref input); + await adderSession.RMWAsync(ref key, ref input); + await adderSession.RMWAsync(ref key, ref input); + + Assert.AreEqual(1, _adder.InitialCount); + Assert.AreEqual(2, _adder.InPlaceCount); + + var empty = default(Empty); + await removerSession.RMWAsync(ref key, ref empty); + + Assert.AreEqual(1, _remover.InPlaceCount); + + var read = await readerSession.ReadAsync(ref key, ref empty); + var result = read.CompleteRead(); + + var actual = result.Item2; + Assert.AreEqual(2, actual.ReferenceCount); + Assert.AreEqual(1000L, actual.Value); + + _faster.Log.FlushAndEvict(true); + + await removerSession.RMWAsync(ref key, ref empty); + read = await readerSession.ReadAsync(ref key, ref empty); + result = read.CompleteRead(); + + actual = result.Item2; + Assert.AreEqual(1, actual.ReferenceCount); + Assert.AreEqual(1000L, actual.Value); + Assert.AreEqual(1, _remover.CopyCount); + } + } + } +} diff --git a/cs/test/GenericLogCompactionTests.cs b/cs/test/GenericLogCompactionTests.cs index c4c1d86a0..445309f95 100644 --- a/cs/test/GenericLogCompactionTests.cs +++ b/cs/test/GenericLogCompactionTests.cs @@ -138,7 +138,6 @@ public void GenericLogCompactionTest2() } } - [Test] public void GenericLogCompactionTest3() { @@ -193,5 +192,135 @@ public void GenericLogCompactionTest3() } } } + + [Test] + public void GenericLogCompactionCustomFunctionsTest1() + { + MyInput input = new MyInput(); + + const int totalRecords = 2000; + var compactUntil = 0L; + + for (var i = 0; i < totalRecords; i++) + { + if (i == totalRecords / 2) + compactUntil = fht.Log.TailAddress; + + var key1 = new MyKey { key = i }; + var value = new MyValue { value = i }; + session.Upsert(ref key1, ref value, 0, 0); + } + + fht.Log.Compact(default(EvenCompactionFunctions), compactUntil); + Assert.IsTrue(fht.Log.BeginAddress == compactUntil); + + // Read 2000 keys - all should be present + for (var i = 0; i < totalRecords; i++) + { + var output = new MyOutput(); + var key1 = new MyKey { key = i }; + var value = new MyValue { value = i }; + + var ctx = (i < (totalRecords / 2) && (i % 2 != 0)) ? 1 : 0; + + var status = session.Read(ref key1, ref input, ref output, ctx, 0); + if (status == Status.PENDING) + { + session.CompletePending(true); + } + else + { + if (ctx == 0) + { + Assert.IsTrue(status == Status.OK); + Assert.IsTrue(output.value.value == value.value); + } + else + { + Assert.IsTrue(status == Status.NOTFOUND); + } + } + } + } + + [Test] + public void GenericLogCompactionCustomFunctionsTest2() + { + // This test checks if CopyInPlace returning false triggers call to Copy + + using var session = fht.NewSession(); + + var key = new MyKey { key = 100 }; + var value = new MyValue { value = 20 }; + + session.Upsert(ref key, ref value, 0, 0); + + fht.Log.Flush(true); + + value = new MyValue { value = 21 }; + session.Upsert(ref key, ref value, 0, 0); + + fht.Log.Flush(true); + + var compactionFunctions = new Test2CompactionFunctions(); + fht.Log.Compact(compactionFunctions, fht.Log.TailAddress); + + Assert.IsTrue(compactionFunctions.CopyCalled); + + var input = default(MyInput); + var output = default(MyOutput); + var status = session.Read(ref key, ref input, ref output, 0, 0); + if (status == Status.PENDING) + { + session.CompletePending(true); + } + else + { + Assert.IsTrue(status == Status.OK); + Assert.IsTrue(output.value.value == value.value); + } + } + + private class Test2CompactionFunctions : ICompactionFunctions + { + public bool CopyCalled; + + public void Copy(ref MyValue src, ref MyValue dst, IVariableLengthStruct valueLength) + { + if (src.value == 21) + CopyCalled = true; + dst = src; + } + + public bool CopyInPlace(ref MyValue src, ref MyValue dst, IVariableLengthStruct valueLength) + { + return false; + } + + public bool IsDeleted(in MyKey key, in MyValue value) + { + return false; + } + } + + private struct EvenCompactionFunctions : ICompactionFunctions + { + public void Copy(ref MyValue src, ref MyValue dst, IVariableLengthStruct valueLength) + { + dst = src; + } + + public bool CopyInPlace(ref MyValue src, ref MyValue dst, IVariableLengthStruct valueLength) + { + dst = src; + return true; + } + + public bool IsDeleted(in MyKey key, in MyValue value) + { + return value.value % 2 != 0; + } + } + } } diff --git a/cs/test/ObjectTestTypes.cs b/cs/test/ObjectTestTypes.cs index 3e68c7c8f..d36451be2 100644 --- a/cs/test/ObjectTestTypes.cs +++ b/cs/test/ObjectTestTypes.cs @@ -161,6 +161,9 @@ public void CopyUpdater(ref MyKey key, ref MyInput input, ref MyValue oldValue, public void ConcurrentReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst) { + if (dst == null) + dst = new MyOutput(); + dst.value = value; } @@ -205,6 +208,9 @@ public void DeleteCompletionCallback(ref MyKey key, int ctx) public void SingleReader(ref MyKey key, ref MyInput input, ref MyValue value, ref MyOutput dst) { + if (dst == null) + dst = new MyOutput(); + dst.value = value; } diff --git a/cs/test/VLTestTypes.cs b/cs/test/VLTestTypes.cs index 3edbcbd5a..858fc4eca 100644 --- a/cs/test/VLTestTypes.cs +++ b/cs/test/VLTestTypes.cs @@ -34,6 +34,11 @@ public int GetLength(ref Key t) return sizeof(long); } + public int GetLength(ref Key t, ref Input input) + { + return sizeof(long); + } + public int GetAverageLength() { return sizeof(long); @@ -56,12 +61,12 @@ public unsafe struct VLValue : IFasterEqualityComparer, IVariableLength public int GetAverageLength() { - return 2*sizeof(int); + return 2 * sizeof(int); } public int GetInitialLength(ref Input input) { - return 2*sizeof(int); + return 2 * sizeof(int); } public int GetLength(ref VLValue t) @@ -69,6 +74,11 @@ public int GetLength(ref VLValue t) return sizeof(int) * t.length; } + public int GetLength(ref VLValue t, ref Input input) + { + return sizeof(int) * t.length; + } + public void ToIntArray(ref int[] dst) { dst = new int[length]; @@ -120,7 +130,7 @@ public void RMWCompletionCallback(ref Key key, ref Input input, Empty ctx, Statu public void ReadCompletionCallback(ref Key key, ref Input input, ref int[] output, Empty ctx, Status status) { Assert.IsTrue(status == Status.OK); - for (int i=0; i