From cbbf3095c00f4e7c3fd338f7e451ea25c6934ab1 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sat, 9 Oct 2021 17:06:50 +0000 Subject: [PATCH 01/33] Checkpoint work --- cs/samples/FasterLogPubSub/Program.cs | 2 +- cs/src/core/Allocator/AllocatorBase.cs | 6 +++-- .../DeviceLogCommitCheckpointManager.cs | 17 +++++++----- cs/src/core/Index/FasterLog/CommitInfo.cs | 10 +++++++ cs/src/core/Index/FasterLog/FasterLog.cs | 16 +++++++----- .../Index/FasterLog/FasterLogRecoveryInfo.cs | 26 +++++++++++++++++++ .../core/Index/FasterLog/ILogCommitManager.cs | 3 ++- .../Index/FasterLog/LocalLogCommitManager.cs | 4 ++- cs/test/FasterLogRecoverReadOnlyTests.cs | 2 +- cs/test/FasterLogTests.cs | 6 ++--- cs/test/RecoverReadOnlyTest.cs | 2 +- 11 files changed, 70 insertions(+), 24 deletions(-) diff --git a/cs/samples/FasterLogPubSub/Program.cs b/cs/samples/FasterLogPubSub/Program.cs index 4446c393e..4d430dc77 100644 --- a/cs/samples/FasterLogPubSub/Program.cs +++ b/cs/samples/FasterLogPubSub/Program.cs @@ -78,7 +78,7 @@ static async Task CommitterAsync(FasterLog log, CancellationToken cancellationTo Console.WriteLine("Committing..."); - await log.CommitAsync(cancellationToken); + await log.CommitAsync(token: cancellationToken); } } catch (OperationCanceledException) { } diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index 37f745f50..6e0194f73 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1124,7 +1124,7 @@ public long TryAllocateRetryNow(int numSlots = 1) /// /// /// - public bool ShiftReadOnlyToTail(out long tailAddress, out SemaphoreSlim notifyDone) + public bool ShiftReadOnlyToTail(out long tailAddress, out SemaphoreSlim notifyDone, long version = -1) { notifyDone = null; tailAddress = GetTailAddress(); @@ -1416,9 +1416,11 @@ protected void ShiftFlushedUntilAddress() FlushCallback?.Invoke( new CommitInfo { + Version = -1, FromAddress = oldFlushedUntilAddress, UntilAddress = currentFlushedUntilAddress, - ErrorCode = errorCode + ErrorCode = errorCode, + Cookie = null }); this.FlushEvent.Set(); diff --git a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs index cd06ce56c..db6bc5b0b 100644 --- a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs +++ b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs @@ -99,9 +99,10 @@ public DeviceLogCommitCheckpointManager(INamedDeviceFactory deviceFactory, strin #region ILogCommitManager /// - public unsafe void Commit(long beginAddress, long untilAddress, byte[] commitMetadata) + public unsafe void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum = -1) { - var device = NextCommitDevice(); + Debug.Assert(!overwriteLogCommits || overwriteLogCommits && proposedCommitNum == -1); + var device = NextCommitDevice(proposedCommitNum); if (device == null) return; @@ -176,14 +177,15 @@ public byte[] GetCommitMetadata(long commitNum) return new Span(body).Slice(sizeof(int)).ToArray(); } - private IDevice NextCommitDevice() + private IDevice NextCommitDevice(long proposedCommitNum) { + var actualNum = proposedCommitNum == -1 ? commitNum + 1 : proposedCommitNum; if (overwriteLogCommits) { if (_disposed) return null; if (singleLogCommitDevice == null) { - singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); + singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(proposedCommitNum)); if (_disposed) { singleLogCommitDevice?.Dispose(); @@ -194,11 +196,12 @@ private IDevice NextCommitDevice() return singleLogCommitDevice; } - return deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum++)); + var result = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(actualNum)); + commitNum = actualNum; + return result; } #endregion - - + #region ICheckpointManager /// public unsafe void CommitIndexCheckpoint(Guid indexToken, byte[] commitMetadata) diff --git a/cs/src/core/Index/FasterLog/CommitInfo.cs b/cs/src/core/Index/FasterLog/CommitInfo.cs index b374c505b..d81305f4e 100644 --- a/cs/src/core/Index/FasterLog/CommitInfo.cs +++ b/cs/src/core/Index/FasterLog/CommitInfo.cs @@ -12,6 +12,11 @@ namespace FASTER.core /// public struct CommitInfo { + /// + /// Version of the commit + /// + public long Version; + /// /// From address of commit range /// @@ -26,6 +31,11 @@ public struct CommitInfo /// Error code (0 = success) /// public uint ErrorCode; + + /// + /// User specified commit cookie + /// + public byte[] Cookie; } /// diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 99a3536db..f2696d0e5 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -472,11 +472,11 @@ public void Commit(bool spinWait = false) /// ongoing commit fails. /// /// - public async ValueTask CommitAsync(CancellationToken token = default) + public async ValueTask CommitAsync(long version = -1, CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(); + var tailAddress = CommitInternal(version); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -494,11 +494,11 @@ public async ValueTask CommitAsync(CancellationToken token = default) /// from prevCommitTask to current fails. /// /// - public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default) + public async ValueTask> CommitAsync(Task prevCommitTask, long version = -1, CancellationToken token = default) { token.ThrowIfCancellationRequested(); if (prevCommitTask == null) prevCommitTask = CommitTask; - var tailAddress = CommitInternal(); + var tailAddress = CommitInternal(version); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -941,13 +941,14 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) FasterLogRecoveryInfo info = new FasterLogRecoveryInfo { BeginAddress = BeginAddress, - FlushedUntilAddress = commitInfo.UntilAddress + FlushedUntilAddress = commitInfo.UntilAddress, + Cookie = commitInfo.Cookie }; // Take snapshot of persisted iterators info.SnapshotIterators(PersistedIterators); - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray()); + logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), commitInfo.Version); LastPersistedIterators = info.Iterators; CommittedBeginAddress = info.BeginAddress; @@ -1052,7 +1053,7 @@ private void SignalWaitingROIterators() // Update commit to release pending iterators. var lci = new LinkedCommitInfo { - CommitInfo = new CommitInfo { FromAddress = BeginAddress, UntilAddress = FlushedUntilAddress }, + CommitInfo = new CommitInfo { Version = -1, FromAddress = BeginAddress, UntilAddress = FlushedUntilAddress }, NextTask = commitTcs.Task }; _commitTcs?.TrySetResult(lci); @@ -1358,6 +1359,7 @@ private long CommitInternal(bool spinWait = false) Interlocked.Increment(ref commitMetadataVersion); CommitCallback(new CommitInfo { + Version = version, FromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, UntilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, ErrorCode = 0 diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index a6b2e80fc..0a880b2a8 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -31,6 +31,11 @@ internal struct FasterLogRecoveryInfo /// public Dictionary Iterators; + /// + /// User-specified commit cookie + /// + public byte[] Cookie; + /// /// Initialize /// @@ -38,6 +43,8 @@ public void Initialize() { BeginAddress = 0; FlushedUntilAddress = 0; + Iterators = null; + Cookie = null; } /// @@ -80,6 +87,15 @@ public void Initialize(BinaryReader reader) Iterators.Add(reader.ReadString(), reader.ReadInt64()); } } + + try + { + count = reader.ReadInt32(); + } + catch { } + + if (count > 0) + Cookie = reader.ReadBytes(count); } /// @@ -115,6 +131,16 @@ public readonly byte[] ToByteArray() { writer.Write(0); } + + if (Cookie != null) + { + writer.Write(Cookie.Length); + writer.Write(Cookie); + } + else + { + writer.Write(0); + } } return ms.ToArray(); } diff --git a/cs/src/core/Index/FasterLog/ILogCommitManager.cs b/cs/src/core/Index/FasterLog/ILogCommitManager.cs index d36066ad0..dd621569b 100644 --- a/cs/src/core/Index/FasterLog/ILogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/ILogCommitManager.cs @@ -18,7 +18,8 @@ public interface ILogCommitManager : IDisposable /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - should be persisted - void Commit(long beginAddress, long untilAddress, byte[] commitMetadata); + /// Proposed commit num, or -1 if none. Caller must ensure commit num supplied is unique and monotonically increasing + void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum = -1); /// /// Return commit metadata diff --git a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs index f47bd16bc..dda28e392 100644 --- a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs @@ -28,7 +28,9 @@ public LocalLogCommitManager(string commitFile) /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata) + /// Ignored param + + public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum = -1) { // Two phase to ensure we write metadata in single Write operation using MemoryStream ms = new(); diff --git a/cs/test/FasterLogRecoverReadOnlyTests.cs b/cs/test/FasterLogRecoverReadOnlyTests.cs index 734e6bc2c..7d96461e7 100644 --- a/cs/test/FasterLogRecoverReadOnlyTests.cs +++ b/cs/test/FasterLogRecoverReadOnlyTests.cs @@ -80,7 +80,7 @@ private async Task CommitterAsync(FasterLog log, CancellationToken cancellationT while (!cancellationToken.IsCancellationRequested) { await Task.Delay(TimeSpan.FromMilliseconds(CommitPeriodMs), cancellationToken); - await log.CommitAsync(cancellationToken); + await log.CommitAsync(token: cancellationToken); } } catch (OperationCanceledException) { } } diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index b3918bc86..a52ac337e 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -164,9 +164,9 @@ protected static async Task LogWriterAsync(FasterLog log, byte[] entry) // Enter in some entries then wait on this separate thread await log.EnqueueAsync(entry); await log.EnqueueAsync(entry); - var commitTask = await log.CommitAsync(null, token); + var commitTask = await log.CommitAsync(null, token: token); await log.EnqueueAsync(entry); - await log.CommitAsync(commitTask, token); + await log.CommitAsync(commitTask, token: token); } } @@ -280,7 +280,7 @@ public async ValueTask TryEnqueue1([Values] LogChecksumType logChecksum, [Values } Assert.IsFalse(waitingReader.IsCompleted); - await log.CommitAsync(token); + await log.CommitAsync(token: token); while (!waitingReader.IsCompleted) ; Assert.IsTrue(waitingReader.IsCompleted); diff --git a/cs/test/RecoverReadOnlyTest.cs b/cs/test/RecoverReadOnlyTest.cs index ed81c90b4..5ecac989c 100644 --- a/cs/test/RecoverReadOnlyTest.cs +++ b/cs/test/RecoverReadOnlyTest.cs @@ -89,7 +89,7 @@ static async Task CommitterAsync(FasterLog log, CancellationToken cancellationTo while (!cancellationToken.IsCancellationRequested) { await Task.Delay(TimeSpan.FromMilliseconds(commitPeriodMs), cancellationToken); - await log.CommitAsync(cancellationToken); + await log.CommitAsync(token: cancellationToken); } } From 82ac15d30785b4bba95f0120c61f7c7fba09ca34 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 10 Oct 2021 00:45:48 +0000 Subject: [PATCH 02/33] Add notion of manual commit --- cs/src/core/Allocator/AllocatorBase.cs | 4 +- cs/src/core/Index/FasterLog/CommitInfo.cs | 10 -- cs/src/core/Index/FasterLog/FasterLog.cs | 116 +++++++++++++++--- .../core/Index/FasterLog/FasterLogSettings.cs | 6 + 4 files changed, 106 insertions(+), 30 deletions(-) diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index 6e0194f73..c1834f3aa 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1416,11 +1416,9 @@ protected void ShiftFlushedUntilAddress() FlushCallback?.Invoke( new CommitInfo { - Version = -1, FromAddress = oldFlushedUntilAddress, UntilAddress = currentFlushedUntilAddress, - ErrorCode = errorCode, - Cookie = null + ErrorCode = errorCode }); this.FlushEvent.Set(); diff --git a/cs/src/core/Index/FasterLog/CommitInfo.cs b/cs/src/core/Index/FasterLog/CommitInfo.cs index d81305f4e..b374c505b 100644 --- a/cs/src/core/Index/FasterLog/CommitInfo.cs +++ b/cs/src/core/Index/FasterLog/CommitInfo.cs @@ -12,11 +12,6 @@ namespace FASTER.core /// public struct CommitInfo { - /// - /// Version of the commit - /// - public long Version; - /// /// From address of commit range /// @@ -31,11 +26,6 @@ public struct CommitInfo /// Error code (0 = success) /// public uint ErrorCode; - - /// - /// User specified commit cookie - /// - public byte[] Cookie; } /// diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index f2696d0e5..a3bb30a10 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -16,6 +16,19 @@ namespace FASTER.core { + internal class ManualCheckpointRequest + { + internal byte[] cookie; + internal long address, version; + internal TaskCompletionSource completed; + public ManualCheckpointRequest(byte[] cookie, long version) + { + this.cookie = cookie; + address = -1; + this.version = version; + } + } + /// /// FASTER log /// @@ -37,6 +50,9 @@ private TaskCompletionSource commitTcs private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + private readonly bool autoCommitOnFlush; + private ManualCheckpointRequest currentCheckpointRequest; + /// /// Beginning address of log /// @@ -167,6 +183,7 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) allocator.HeadAddress = long.MaxValue; } + autoCommitOnFlush = false; } /// @@ -466,17 +483,61 @@ public void Commit(bool spinWait = false) CommitInternal(spinWait); } + /// + /// Manually trigger a checkpoint at version v and persist with it the supplied cookie, when autoCommitOnFlush is + /// turned off. This is an advanced functionality, and user is expected to be responsible for supplying unique + /// and monotonically increasing version numbers. + /// + /// version of the checkpoint + /// user-specified cookie + /// whether the commit call failed and should be retried later + public bool ManualCommitWithCookie(long version, byte[] cookie) + { + if (autoCommitOnFlush) + throw new FasterException( + "Checkpointing with cookie and custom versioning is only supported when auto commit is turned off"); + + var newRequest = new ManualCheckpointRequest(cookie, version); + // Fail when a concurrent checkpoint is underway + if (Interlocked.CompareExchange(ref currentCheckpointRequest, newRequest, null) != null) + return false; + + if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) + { + newRequest.address = tailAddress; + epoch.Suspend(); + } + else + { + // May need to commit begin address and/or iterators + epoch.Suspend(); + var beginAddress = allocator.BeginAddress; + newRequest.address = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + if (beginAddress > CommittedBeginAddress || IteratorsChanged()) + { + Interlocked.Increment(ref commitMetadataVersion); + CommitCallback(new CommitInfo + { + FromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, + UntilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, + ErrorCode = 0, + }); + } + } + return true; + } + /// /// Async commit log (until tail), completes only when we /// complete the commit. Throws exception if this or any /// ongoing commit fails. /// /// - public async ValueTask CommitAsync(long version = -1, CancellationToken token = default) + public async ValueTask CommitAsync(CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(version); + var tailAddress = CommitInternal(); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -494,11 +555,11 @@ public async ValueTask CommitAsync(long version = -1, CancellationToken token = /// from prevCommitTask to current fails. /// /// - public async ValueTask> CommitAsync(Task prevCommitTask, long version = -1, CancellationToken token = default) + public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default) { token.ThrowIfCancellationRequested(); if (prevCommitTask == null) prevCommitTask = CommitTask; - var tailAddress = CommitInternal(version); + var tailAddress = CommitInternal(); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -916,17 +977,39 @@ private int Align(int length) /// private void CommitCallback(CommitInfo commitInfo) { + // Ignore any potential auto flushes when there's no active checkpoint task + if (!autoCommitOnFlush && currentCheckpointRequest == null) return; + commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } private void SerialCommitCallbackWorker(CommitInfo commitInfo) { - // Check if commit is already covered - if (CommittedBeginAddress >= BeginAddress && - CommittedUntilAddress >= commitInfo.UntilAddress && - persistedCommitMetadataVersion >= commitMetadataVersion && - commitInfo.ErrorCode == 0) - return; + long version = -1; + byte[] cookie = null; + if (!autoCommitOnFlush) + { + Debug.Assert(currentCheckpointRequest != null); + // Wait for checkpoint request address to become known to avoid missing the address + while (currentCheckpointRequest.address == -1) + Thread.Yield(); + // Return if this is a flush that corresponds to the manual commit + if (commitInfo.UntilAddress != currentCheckpointRequest.address) return; + // Fetch the associated cookie and version information, and unblock any potential concurrent checkpoint + // requests + version = currentCheckpointRequest.version; + cookie = currentCheckpointRequest.cookie; + currentCheckpointRequest = null; + } + else + { + // Check if commit is already covered + if (CommittedBeginAddress >= BeginAddress && + CommittedUntilAddress >= commitInfo.UntilAddress && + persistedCommitMetadataVersion >= commitMetadataVersion && + commitInfo.ErrorCode == 0) + return; + } if (commitInfo.ErrorCode == 0) { @@ -937,18 +1020,18 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) commitInfo.FromAddress = CommittedUntilAddress; if (CommittedUntilAddress > commitInfo.UntilAddress) commitInfo.UntilAddress = CommittedUntilAddress; - + FasterLogRecoveryInfo info = new FasterLogRecoveryInfo { BeginAddress = BeginAddress, FlushedUntilAddress = commitInfo.UntilAddress, - Cookie = commitInfo.Cookie + Cookie = cookie }; // Take snapshot of persisted iterators info.SnapshotIterators(PersistedIterators); - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), commitInfo.Version); + logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), version); LastPersistedIterators = info.Iterators; CommittedBeginAddress = info.BeginAddress; @@ -1053,7 +1136,7 @@ private void SignalWaitingROIterators() // Update commit to release pending iterators. var lci = new LinkedCommitInfo { - CommitInfo = new CommitInfo { Version = -1, FromAddress = BeginAddress, UntilAddress = FlushedUntilAddress }, + CommitInfo = new CommitInfo { FromAddress = BeginAddress, UntilAddress = FlushedUntilAddress }, NextTask = commitTcs.Task }; _commitTcs?.TrySetResult(lci); @@ -1335,7 +1418,7 @@ private long CommitInternal(bool spinWait = false) { if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); - + epoch.Resume(); if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) { @@ -1359,10 +1442,9 @@ private long CommitInternal(bool spinWait = false) Interlocked.Increment(ref commitMetadataVersion); CommitCallback(new CommitInfo { - Version = version, FromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, UntilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, - ErrorCode = 0 + ErrorCode = 0, }); } } diff --git a/cs/src/core/Index/FasterLog/FasterLogSettings.cs b/cs/src/core/Index/FasterLog/FasterLogSettings.cs index 9fe7cc8e5..361f5cb72 100644 --- a/cs/src/core/Index/FasterLog/FasterLogSettings.cs +++ b/cs/src/core/Index/FasterLog/FasterLogSettings.cs @@ -89,6 +89,12 @@ public class FasterLogSettings /// public bool ReadOnlyMode = false; + /// + /// When set to true --- automatically create a commit for every disk flush (which may happen in the background). + /// Otherwise, a commit will only be created and recoverable where Commit() is invoked manually + /// + public bool AutoCommitOnFlush = true; + internal LogSettings GetLogSettings() { return new LogSettings From fb863a20e2670693686aa222185a9b367e00871f Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 10 Oct 2021 00:48:59 +0000 Subject: [PATCH 03/33] fix --- cs/src/core/Index/FasterLog/FasterLog.cs | 1 - 1 file changed, 1 deletion(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index a3bb30a10..2f2a5befb 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -20,7 +20,6 @@ internal class ManualCheckpointRequest { internal byte[] cookie; internal long address, version; - internal TaskCompletionSource completed; public ManualCheckpointRequest(byte[] cookie, long version) { this.cookie = cookie; From aaf0f3b31a6ce291b22f4e4b74cd70e703f481c0 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 10 Oct 2021 01:02:16 +0000 Subject: [PATCH 04/33] Recovery of commit cookie --- cs/src/core/Index/FasterLog/FasterLog.cs | 51 +++++++++++++++++------- 1 file changed, 37 insertions(+), 14 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 2f2a5befb..bd8368918 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -87,6 +87,11 @@ private TaskCompletionSource refreshUncommittedTcs /// public long CommittedBeginAddress; + /// + /// Recovered Commit Cookie + /// + public byte[] RecoveredCookie; + /// /// Task notifying commit completions /// @@ -130,10 +135,11 @@ internal readonly ConcurrentDictionary PersistedI /// Create new log instance /// /// - public FasterLog(FasterLogSettings logSettings) + public FasterLog(FasterLogSettings logSettings, long requestedVersion = -1) : this(logSettings, false) { - this.RecoveredIterators = Restore(); + Restore(out var it, out RecoveredCookie, requestedVersion); + RecoveredIterators = it; } /// @@ -994,8 +1000,8 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) Thread.Yield(); // Return if this is a flush that corresponds to the manual commit if (commitInfo.UntilAddress != currentCheckpointRequest.address) return; - // Fetch the associated cookie and version information, and unblock any potential concurrent checkpoint - // requests + // Fetch the associated cookie and version information, and unblock any potential concurrent + // checkpoint requests version = currentCheckpointRequest.version; cookie = currentCheckpointRequest.cookie; currentCheckpointRequest = null; @@ -1105,7 +1111,7 @@ public void RecoverReadOnly() if (!readOnlyMode) throw new FasterException("This method can only be used with a read-only FasterLog instance used for iteration. Set FasterLogSettings.ReadOnlyMode to true during creation to indicate this."); - this.Restore(); + this.Restore(out _, out _); SignalWaitingROIterators(); } @@ -1144,24 +1150,41 @@ private void SignalWaitingROIterators() /// /// Restore log synchronously /// - private Dictionary Restore() + private bool Restore(out Dictionary iterators, out byte[] cookie, long requestedVersion = -1) { + iterators = null; + cookie = null; + if (requestedVersion != -1) + return RestoreFromCommitNum(requestedVersion, out iterators, out cookie); + foreach (var commitNum in logCommitManager.ListCommits()) { try { - if (!PrepareToRestoreFromCommit(commitNum, out FasterLogRecoveryInfo info, out long headAddress)) - return default; - - if (headAddress > 0) - allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.FlushedUntilAddress, info.FlushedUntilAddress); - - return CompleteRestoreFromCommit(info); + return RestoreFromCommitNum(requestedVersion, out iterators, out cookie); } catch { } } Debug.WriteLine("Unable to recover using any available commit"); - return null; + return false; + } + + private bool RestoreFromCommitNum(long commitNum, out Dictionary iterators, out byte[] cookie) + { + iterators = null; + cookie = null; + if (!PrepareToRestoreFromCommit(commitNum, out var info, out var headAddress)) + { + Debug.WriteLine("Unable to recover using any specified commit num: " + commitNum); + return false; + } + + if (headAddress > 0) + allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.FlushedUntilAddress, info.FlushedUntilAddress); + + iterators = CompleteRestoreFromCommit(info); + cookie = info.Cookie; + return true; } /// From 1e2e8b9534e48ab39b48286ceef299f10063a72e Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Tue, 12 Oct 2021 20:22:04 +0000 Subject: [PATCH 05/33] alternative design of log-cookie --- cs/src/core/Index/FasterLog/FasterLog.cs | 158 ++++++++++------------- 1 file changed, 71 insertions(+), 87 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index bd8368918..bbc5c1a71 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -19,12 +19,14 @@ namespace FASTER.core internal class ManualCheckpointRequest { internal byte[] cookie; - internal long address, version; - public ManualCheckpointRequest(byte[] cookie, long version) + internal long commitNum; + internal TaskCompletionSource checkpointTcs; + + public ManualCheckpointRequest(byte[] cookie, long commitNum) { this.cookie = cookie; - address = -1; - this.version = version; + this.commitNum = commitNum; + checkpointTcs = new TaskCompletionSource(); } } @@ -50,7 +52,7 @@ private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); private readonly bool autoCommitOnFlush; - private ManualCheckpointRequest currentCheckpointRequest; + private ConcurrentDictionary manualCheckpointRequests; /// /// Beginning address of log @@ -189,6 +191,7 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) } autoCommitOnFlush = false; + manualCheckpointRequests = new ConcurrentDictionary(); } /// @@ -478,6 +481,12 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok #region Commit and CommitAsync + public Task CommitWithCookie(byte[] cookie, long commitNum = -1, long cookieLowerAddr = -1) + { + CommitInternal(out var task, commitNum, cookie, cookieLowerAddr); + return task; + } + /// /// Issue commit request for log (until tail) /// @@ -485,51 +494,12 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok /// public void Commit(bool spinWait = false) { - CommitInternal(spinWait); - } - - /// - /// Manually trigger a checkpoint at version v and persist with it the supplied cookie, when autoCommitOnFlush is - /// turned off. This is an advanced functionality, and user is expected to be responsible for supplying unique - /// and monotonically increasing version numbers. - /// - /// version of the checkpoint - /// user-specified cookie - /// whether the commit call failed and should be retried later - public bool ManualCommitWithCookie(long version, byte[] cookie) - { - if (autoCommitOnFlush) - throw new FasterException( - "Checkpointing with cookie and custom versioning is only supported when auto commit is turned off"); - - var newRequest = new ManualCheckpointRequest(cookie, version); - // Fail when a concurrent checkpoint is underway - if (Interlocked.CompareExchange(ref currentCheckpointRequest, newRequest, null) != null) - return false; - - if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) - { - newRequest.address = tailAddress; - epoch.Suspend(); - } - else + CommitInternal(out var task); + if (spinWait) { - // May need to commit begin address and/or iterators - epoch.Suspend(); - var beginAddress = allocator.BeginAddress; - newRequest.address = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; - if (beginAddress > CommittedBeginAddress || IteratorsChanged()) - { - Interlocked.Increment(ref commitMetadataVersion); - CommitCallback(new CommitInfo - { - FromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, - UntilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, - ErrorCode = 0, - }); - } + while (!task.IsCompleted) + Thread.Sleep(1); } - return true; } /// @@ -542,7 +512,7 @@ public async ValueTask CommitAsync(CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(); + var tailAddress = CommitInternal(out _); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -564,7 +534,7 @@ public async ValueTask> CommitAsync(Task private void CommitCallback(CommitInfo commitInfo) { - // Ignore any potential auto flushes when there's no active checkpoint task - if (!autoCommitOnFlush && currentCheckpointRequest == null) return; - commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } private void SerialCommitCallbackWorker(CommitInfo commitInfo) { - long version = -1; - byte[] cookie = null; - if (!autoCommitOnFlush) + var coveredRequests = new List<(long, ManualCheckpointRequest)>(); + foreach (var entry in manualCheckpointRequests) { - Debug.Assert(currentCheckpointRequest != null); - // Wait for checkpoint request address to become known to avoid missing the address - while (currentCheckpointRequest.address == -1) - Thread.Yield(); - // Return if this is a flush that corresponds to the manual commit - if (commitInfo.UntilAddress != currentCheckpointRequest.address) return; - // Fetch the associated cookie and version information, and unblock any potential concurrent - // checkpoint requests - version = currentCheckpointRequest.version; - cookie = currentCheckpointRequest.cookie; - currentCheckpointRequest = null; + if (entry.Key > commitInfo.UntilAddress) break; + coveredRequests.Add(ValueTuple.Create(entry.Key, entry.Value)); } - else + + foreach (var (offset, _) in coveredRequests) + manualCheckpointRequests.TryRemove(offset, out _); + + // AutoCommit is off and there is no outstanding request --- safe to skip + if (coveredRequests.Count == 0 && !autoCommitOnFlush) return; + + // Check if commit is already covered + if (CommittedBeginAddress >= BeginAddress && + CommittedUntilAddress >= commitInfo.UntilAddress && + persistedCommitMetadataVersion >= commitMetadataVersion && + commitInfo.ErrorCode == 0) { - // Check if commit is already covered - if (CommittedBeginAddress >= BeginAddress && - CommittedUntilAddress >= commitInfo.UntilAddress && - persistedCommitMetadataVersion >= commitMetadataVersion && - commitInfo.ErrorCode == 0) - return; + // In this case --- need to mark commit cookies failed because there now exists a commit on disk + // with offset larger than specified but no cookie + foreach(var (_, request) in coveredRequests) + request.checkpointTcs.SetResult(false); + return; } if (commitInfo.ErrorCode == 0) { // Capture CMV first, so metadata prior to CMV update is visible to commit long _localCMV = commitMetadataVersion; + var latestRequest = coveredRequests.Count == 0 ? null : coveredRequests[coveredRequests.Count - 1].Item2; if (CommittedUntilAddress > commitInfo.FromAddress) commitInfo.FromAddress = CommittedUntilAddress; @@ -1030,13 +998,13 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) { BeginAddress = BeginAddress, FlushedUntilAddress = commitInfo.UntilAddress, - Cookie = cookie + Cookie = latestRequest?.cookie }; // Take snapshot of persisted iterators info.SnapshotIterators(PersistedIterators); - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), version); + logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), latestRequest?.commitNum ?? -1); LastPersistedIterators = info.Iterators; CommittedBeginAddress = info.BeginAddress; @@ -1046,6 +1014,15 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) // Update completed address for persisted iterators info.CommitIterators(PersistedIterators); + // Mark all commit requests completedc + foreach(var (_, request) in coveredRequests) + request.checkpointTcs.SetResult(true); + } + else + { + // Mark commits failed + foreach(var (_, request) in coveredRequests) + request.checkpointTcs.SetResult(false); } var _commitTcs = commitTcs; @@ -1161,7 +1138,7 @@ private bool Restore(out Dictionary iterators, out byte[] cookie, { try { - return RestoreFromCommitNum(requestedVersion, out iterators, out cookie); + return RestoreFromCommitNum(commitNum, out iterators, out cookie); } catch { } } @@ -1436,22 +1413,24 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private long CommitInternal(bool spinWait = false) + private long CommitInternal(out Task completionTask, long commitNum = -1, byte[] cookie = null, long cookieLowerAddr = -1) { + completionTask = default; if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); epoch.Resume(); + if (cookieLowerAddr == -1) cookieLowerAddr = allocator.GetTailAddress(); + + // No point in computing a commit when cookie is already out-of-date + if (cookieLowerAddr < CommittedUntilAddress) return -1; + var request = new ManualCheckpointRequest(cookie, commitNum); + // Not expecting to have a lot of concurrency + manualCheckpointRequests.TryAdd(cookieLowerAddr, request); + completionTask = request.checkpointTcs.Task; + if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) { - if (spinWait) - { - while (CommittedUntilAddress < tailAddress) - { - epoch.ProtectAndDrain(); - Thread.Yield(); - } - } epoch.Suspend(); } else @@ -1459,7 +1438,7 @@ private long CommitInternal(bool spinWait = false) // May need to commit begin address and/or iterators epoch.Suspend(); var beginAddress = allocator.BeginAddress; - if (beginAddress > CommittedBeginAddress || IteratorsChanged()) + if (beginAddress > CommittedBeginAddress || IteratorsChanged() || cookie != null) { Interlocked.Increment(ref commitMetadataVersion); CommitCallback(new CommitInfo @@ -1469,6 +1448,11 @@ private long CommitInternal(bool spinWait = false) ErrorCode = 0, }); } + else + { + manualCheckpointRequests.TryRemove(cookieLowerAddr, out _); + request.checkpointTcs.SetResult(true); + } } return tailAddress; From 759e70b6d3dafcda2738174689176099c55db023 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 14 Oct 2021 17:45:02 +0000 Subject: [PATCH 06/33] Implementation improvement --- .../DeviceLogCommitCheckpointManager.cs | 4 +- cs/src/core/Index/FasterLog/FasterLog.cs | 171 +++++++++++------- 2 files changed, 107 insertions(+), 68 deletions(-) diff --git a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs index db6bc5b0b..16f06181b 100644 --- a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs +++ b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs @@ -179,13 +179,12 @@ public byte[] GetCommitMetadata(long commitNum) private IDevice NextCommitDevice(long proposedCommitNum) { - var actualNum = proposedCommitNum == -1 ? commitNum + 1 : proposedCommitNum; if (overwriteLogCommits) { if (_disposed) return null; if (singleLogCommitDevice == null) { - singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(proposedCommitNum)); + singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); if (_disposed) { singleLogCommitDevice?.Dispose(); @@ -196,6 +195,7 @@ private IDevice NextCommitDevice(long proposedCommitNum) return singleLogCommitDevice; } + var actualNum = proposedCommitNum == -1 ? commitNum + 1 : proposedCommitNum; var result = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(actualNum)); commitNum = actualNum; return result; diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index bbc5c1a71..af13b05e5 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -7,6 +7,7 @@ using System.Buffers; using System.Collections.Concurrent; using System.Collections.Generic; +using System.ComponentModel.Design; using System.Diagnostics; using System.IO; using System.Linq; @@ -16,20 +17,27 @@ namespace FASTER.core { - internal class ManualCheckpointRequest + internal struct ManualCheckpointRequest { - internal byte[] cookie; - internal long commitNum; - internal TaskCompletionSource checkpointTcs; + private long proposedCommitNum; + + internal bool IsSet() => proposedCommitNum != 0; - public ManualCheckpointRequest(byte[] cookie, long commitNum) + internal long DrainCommitNum() + { + var result = Interlocked.Exchange(ref proposedCommitNum, 0); + if (result == 0) return -1; + return result; + } + + internal bool TrySetCommitNum(long num) { - this.cookie = cookie; - this.commitNum = commitNum; - checkpointTcs = new TaskCompletionSource(); + return Interlocked.CompareExchange(ref proposedCommitNum, num, 0) == 0; } + } + /// /// FASTER log /// @@ -51,8 +59,14 @@ private TaskCompletionSource commitTcs private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + // Manual vs. Auto commit mode private readonly bool autoCommitOnFlush; - private ConcurrentDictionary manualCheckpointRequests; + private ManualCheckpointRequest currentManualRequest; + + // Commit Cookie Related + private Queue<(long, byte[])> commitCookies; + private long cookieUntilAddr; + private bool cookieChanged; /// /// Beginning address of log @@ -137,10 +151,10 @@ internal readonly ConcurrentDictionary PersistedI /// Create new log instance /// /// - public FasterLog(FasterLogSettings logSettings, long requestedVersion = -1) + public FasterLog(FasterLogSettings logSettings, long requestedCommitNum = -1) : this(logSettings, false) { - Restore(out var it, out RecoveredCookie, requestedVersion); + Restore(out var it, out RecoveredCookie, requestedCommitNum); RecoveredIterators = it; } @@ -190,8 +204,10 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) allocator.HeadAddress = long.MaxValue; } - autoCommitOnFlush = false; - manualCheckpointRequests = new ConcurrentDictionary(); + autoCommitOnFlush = logSettings.AutoCommitOnFlush; + commitCookies = new Queue<(long, byte[])>(); + cookieChanged = false; + currentManualRequest = new ManualCheckpointRequest(); } /// @@ -481,10 +497,28 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok #region Commit and CommitAsync - public Task CommitWithCookie(byte[] cookie, long commitNum = -1, long cookieLowerAddr = -1) + public bool AddCommitCookie(byte[] cookie, long addrLowerBound = -1) { - CommitInternal(out var task, commitNum, cookie, cookieLowerAddr); - return task; + if (addrLowerBound == -1) addrLowerBound = TailAddress; + // Not expected to have heavy contention + lock (commitCookies) + { + // Ensure that cookie is going to be a part of any commit that includes the specified addrLowerBound + if (CommittedUntilAddress >= cookieUntilAddr) return false; + + // Reject any out-of-order commit cookies as they will never be written out + if (cookieUntilAddr > addrLowerBound) return false; + + // Because SerialCommitCallbackWorker will also access the queue under mutex, it is safe to assume + // anything that is enqueued here will be written out later + cookieUntilAddr = addrLowerBound; + commitCookies.Enqueue(ValueTuple.Create(addrLowerBound, cookie)); + + // If associating a new cookie with the same address, mark so commit code path can pick up the change + if (cookieUntilAddr == addrLowerBound) + cookieChanged = true; + } + return true; } /// @@ -494,12 +528,7 @@ public Task CommitWithCookie(byte[] cookie, long commitNum = -1, long cook /// public void Commit(bool spinWait = false) { - CommitInternal(out var task); - if (spinWait) - { - while (!task.IsCompleted) - Thread.Sleep(1); - } + CommitInternal(spinWait); } /// @@ -512,7 +541,7 @@ public async ValueTask CommitAsync(CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(out _); + var tailAddress = CommitInternal(); while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) { @@ -534,7 +563,7 @@ public async ValueTask> CommitAsync(Task> CommitAsync(Task + /// Issue commit request for log (until tail), persisting as the given commitNum. It is up to the users to ensure + /// that the supplied commitNum is unique and monotonically increasing. + /// + /// the commit num to write out + /// whether manual request is successfully triggered. If not, request should be retried later. + public bool CommitManuallyAtNum(long commitNum) + { + if (autoCommitOnFlush) + throw new FasterException("Specifying of commit num is only supported when auto commit is turned off"); + if (!currentManualRequest.TrySetCommitNum(commitNum)) return false; + CommitInternal(); + return true; + } + /// /// Trigger a refresh of information about uncommitted part of log (tail address) to ensure visibility /// to uncommitted scan iterators. Will cause SafeTailAddress to reflect the current tail address. @@ -957,18 +1001,26 @@ private void CommitCallback(CommitInfo commitInfo) private void SerialCommitCallbackWorker(CommitInfo commitInfo) { - var coveredRequests = new List<(long, ManualCheckpointRequest)>(); - foreach (var entry in manualCheckpointRequests) + byte[] cookie = null; + // Not expected to have heavy contention + lock (commitCookies) { - if (entry.Key > commitInfo.UntilAddress) break; - coveredRequests.Add(ValueTuple.Create(entry.Key, entry.Value)); + if (commitCookies.Count != 0) + { + while (true) + { + var (addr, bytes) = commitCookies.Peek(); + cookie = bytes; + if (addr > commitInfo.UntilAddress) break; + if (commitCookies.Count == 1) break; + commitCookies.Dequeue(); + } + cookieChanged = false; + } } - foreach (var (offset, _) in coveredRequests) - manualCheckpointRequests.TryRemove(offset, out _); - // AutoCommit is off and there is no outstanding request --- safe to skip - if (coveredRequests.Count == 0 && !autoCommitOnFlush) return; + if (!autoCommitOnFlush && !currentManualRequest.IsSet()) return; // Check if commit is already covered if (CommittedBeginAddress >= BeginAddress && @@ -976,10 +1028,8 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) persistedCommitMetadataVersion >= commitMetadataVersion && commitInfo.ErrorCode == 0) { - // In this case --- need to mark commit cookies failed because there now exists a commit on disk - // with offset larger than specified but no cookie - foreach(var (_, request) in coveredRequests) - request.checkpointTcs.SetResult(false); + // Should only be a scenario where auto commit is true + Debug.Assert(autoCommitOnFlush); return; } @@ -987,7 +1037,6 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) { // Capture CMV first, so metadata prior to CMV update is visible to commit long _localCMV = commitMetadataVersion; - var latestRequest = coveredRequests.Count == 0 ? null : coveredRequests[coveredRequests.Count - 1].Item2; if (CommittedUntilAddress > commitInfo.FromAddress) commitInfo.FromAddress = CommittedUntilAddress; @@ -998,13 +1047,13 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) { BeginAddress = BeginAddress, FlushedUntilAddress = commitInfo.UntilAddress, - Cookie = latestRequest?.cookie + Cookie = cookie }; // Take snapshot of persisted iterators info.SnapshotIterators(PersistedIterators); - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), latestRequest?.commitNum ?? -1); + logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), currentManualRequest.DrainCommitNum()); LastPersistedIterators = info.Iterators; CommittedBeginAddress = info.BeginAddress; @@ -1012,17 +1061,9 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) if (_localCMV > persistedCommitMetadataVersion) persistedCommitMetadataVersion = _localCMV; + // Update completed address for persisted iterators info.CommitIterators(PersistedIterators); - // Mark all commit requests completedc - foreach(var (_, request) in coveredRequests) - request.checkpointTcs.SetResult(true); - } - else - { - // Mark commits failed - foreach(var (_, request) in coveredRequests) - request.checkpointTcs.SetResult(false); } var _commitTcs = commitTcs; @@ -1413,32 +1454,35 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private long CommitInternal(out Task completionTask, long commitNum = -1, byte[] cookie = null, long cookieLowerAddr = -1) + private long CommitInternal(bool spinWait = false) { - completionTask = default; if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); - + epoch.Resume(); - if (cookieLowerAddr == -1) cookieLowerAddr = allocator.GetTailAddress(); - - // No point in computing a commit when cookie is already out-of-date - if (cookieLowerAddr < CommittedUntilAddress) return -1; - var request = new ManualCheckpointRequest(cookie, commitNum); - // Not expecting to have a lot of concurrency - manualCheckpointRequests.TryAdd(cookieLowerAddr, request); - completionTask = request.checkpointTcs.Task; - + if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) { + if (spinWait) + { + while (CommittedUntilAddress < tailAddress) + { + epoch.ProtectAndDrain(); + Thread.Yield(); + } + } epoch.Suspend(); + } else { // May need to commit begin address and/or iterators epoch.Suspend(); var beginAddress = allocator.BeginAddress; - if (beginAddress > CommittedBeginAddress || IteratorsChanged() || cookie != null) + // Reading cookieChanged concurrently without locking is fine here --- if interleaved with a concurrent + // AddCommitCookie call, the function will not properly trigger a checkpoint, and users will perceive + // it as if AddCommitCookie is ordered after the Commit call. + if (beginAddress > CommittedBeginAddress || IteratorsChanged() || cookieChanged) { Interlocked.Increment(ref commitMetadataVersion); CommitCallback(new CommitInfo @@ -1448,11 +1492,6 @@ private long CommitInternal(out Task completionTask, long commitNum = -1, ErrorCode = 0, }); } - else - { - manualCheckpointRequests.TryRemove(cookieLowerAddr, out _); - request.checkpointTcs.SetResult(true); - } } return tailAddress; From 24fe876b6ef5de767cd6f949fb74dda71b465c91 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Mon, 18 Oct 2021 20:02:59 +0000 Subject: [PATCH 07/33] code review comments --- cs/src/core/Index/FasterLog/FasterLog.cs | 28 +++++++++++++++--------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index af13b05e5..e0dea660f 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -205,7 +205,6 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) } autoCommitOnFlush = logSettings.AutoCommitOnFlush; - commitCookies = new Queue<(long, byte[])>(); cookieChanged = false; currentManualRequest = new ManualCheckpointRequest(); } @@ -500,6 +499,11 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok public bool AddCommitCookie(byte[] cookie, long addrLowerBound = -1) { if (addrLowerBound == -1) addrLowerBound = TailAddress; + + // Lazily allocate data structure + if (commitCookies == null) + Interlocked.CompareExchange(ref commitCookies, new Queue<(long, byte[])>(), null); + // Not expected to have heavy contention lock (commitCookies) { @@ -1003,19 +1007,23 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) { byte[] cookie = null; // Not expected to have heavy contention - lock (commitCookies) + if (commitCookies != null) { - if (commitCookies.Count != 0) + lock (commitCookies) { - while (true) + if (commitCookies.Count != 0) { - var (addr, bytes) = commitCookies.Peek(); - cookie = bytes; - if (addr > commitInfo.UntilAddress) break; - if (commitCookies.Count == 1) break; - commitCookies.Dequeue(); + while (true) + { + var (addr, bytes) = commitCookies.Peek(); + cookie = bytes; + if (addr > commitInfo.UntilAddress) break; + if (commitCookies.Count == 1) break; + commitCookies.Dequeue(); + } + + cookieChanged = false; } - cookieChanged = false; } } From e406a49c3a3cd5714b4af7dde9ac799c7fd47085 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 21 Oct 2021 15:03:41 +0000 Subject: [PATCH 08/33] Cleanup + testing --- cs/src/core/Allocator/AllocatorBase.cs | 2 +- cs/src/core/Index/FasterLog/FasterLog.cs | 28 ++- .../core/Index/FasterLog/FasterLogSettings.cs | 5 +- .../core/Index/FasterLog/ILogCommitManager.cs | 3 +- .../Index/FasterLog/LocalLogCommitManager.cs | 4 +- cs/test/FasterLogTests.cs | 165 +++++++++++++++++- 6 files changed, 194 insertions(+), 13 deletions(-) diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index c1834f3aa..37f745f50 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1124,7 +1124,7 @@ public long TryAllocateRetryNow(int numSlots = 1) /// /// /// - public bool ShiftReadOnlyToTail(out long tailAddress, out SemaphoreSlim notifyDone, long version = -1) + public bool ShiftReadOnlyToTail(out long tailAddress, out SemaphoreSlim notifyDone) { notifyDone = null; tailAddress = GetTailAddress(); diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index e0dea660f..d1314276b 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -7,7 +7,6 @@ using System.Buffers; using System.Collections.Concurrent; using System.Collections.Generic; -using System.ComponentModel.Design; using System.Diagnostics; using System.IO; using System.Linq; @@ -17,6 +16,7 @@ namespace FASTER.core { + // Manual checkpoint request with information about the desired commit num internal struct ManualCheckpointRequest { private long proposedCommitNum; @@ -151,6 +151,7 @@ internal readonly ConcurrentDictionary PersistedI /// Create new log instance /// /// + /// specific commit number to recover from (or -1 for latest) public FasterLog(FasterLogSettings logSettings, long requestedCommitNum = -1) : this(logSettings, false) { @@ -496,6 +497,17 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok #region Commit and CommitAsync + /// + /// Associates the given cookie (an opaque chunk of client-specified bytes) with a log address (or -1 for + /// current tail). If the method returns true, the cookie will be recovered in RecoveredCookie as part of any + /// commit including the log address, unless overwritten by a later cookie. + /// + /// Thread-safe. Generally, cookies associated with larger address are considered more recent. Behavior is + /// arbitrary if multiple cookies are associated with the same address concurrently. + /// + /// custom byte chunks to persist with commits + /// address to associate cookie with, or -1 for current tail + /// whether the cookie will be present with all future commits including the lower bound public bool AddCommitCookie(byte[] cookie, long addrLowerBound = -1) { if (addrLowerBound == -1) addrLowerBound = TailAddress; @@ -508,7 +520,7 @@ public bool AddCommitCookie(byte[] cookie, long addrLowerBound = -1) lock (commitCookies) { // Ensure that cookie is going to be a part of any commit that includes the specified addrLowerBound - if (CommittedUntilAddress >= cookieUntilAddr) return false; + if (CommittedUntilAddress >= addrLowerBound) return false; // Reject any out-of-order commit cookies as they will never be written out if (cookieUntilAddr > addrLowerBound) return false; @@ -586,13 +598,14 @@ public async ValueTask> CommitAsync(Task /// the commit num to write out + /// spin until commit is complete /// whether manual request is successfully triggered. If not, request should be retried later. - public bool CommitManuallyAtNum(long commitNum) + public bool CommitManuallyAtNum(long commitNum, bool spinWait = false) { if (autoCommitOnFlush) throw new FasterException("Specifying of commit num is only supported when auto commit is turned off"); if (!currentManualRequest.TrySetCommitNum(commitNum)) return false; - CommitInternal(); + CommitInternal(spinWait); return true; } @@ -1016,8 +1029,8 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) while (true) { var (addr, bytes) = commitCookies.Peek(); - cookie = bytes; if (addr > commitInfo.UntilAddress) break; + cookie = bytes; if (commitCookies.Count == 1) break; commitCookies.Dequeue(); } @@ -1462,13 +1475,16 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private long CommitInternal(bool spinWait = false) + private long CommitInternal(bool spinWait = false, long commitNum = -1) { if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); epoch.Resume(); + if (autoCommitOnFlush == false) + currentManualRequest.TrySetCommitNum(commitNum); + if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) { if (spinWait) diff --git a/cs/src/core/Index/FasterLog/FasterLogSettings.cs b/cs/src/core/Index/FasterLog/FasterLogSettings.cs index 361f5cb72..c45107346 100644 --- a/cs/src/core/Index/FasterLog/FasterLogSettings.cs +++ b/cs/src/core/Index/FasterLog/FasterLogSettings.cs @@ -90,8 +90,9 @@ public class FasterLogSettings public bool ReadOnlyMode = false; /// - /// When set to true --- automatically create a commit for every disk flush (which may happen in the background). - /// Otherwise, a commit will only be created and recoverable where Commit() is invoked manually + /// When set to true --- automatically create a commit for every disk flush (which may happen in the background + /// due to enqueue operations). Otherwise, a commit will only be created and recoverable where Commit() is \ + /// invoked manually /// public bool AutoCommitOnFlush = true; diff --git a/cs/src/core/Index/FasterLog/ILogCommitManager.cs b/cs/src/core/Index/FasterLog/ILogCommitManager.cs index dd621569b..015ae1468 100644 --- a/cs/src/core/Index/FasterLog/ILogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/ILogCommitManager.cs @@ -18,7 +18,8 @@ public interface ILogCommitManager : IDisposable /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - should be persisted - /// Proposed commit num, or -1 if none. Caller must ensure commit num supplied is unique and monotonically increasing + /// Proposed commit num, or -1 if none, in which case the implementation will pick one. + /// Caller must ensure commit num supplied is unique and monotonically increasing void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum = -1); /// diff --git a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs index dda28e392..e803a4898 100644 --- a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs @@ -28,10 +28,12 @@ public LocalLogCommitManager(string commitFile) /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - /// Ignored param + /// Ignored param public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum = -1) { + if (proposedCommitNum != -1) + throw new FasterException("LocalLogCommitManager does not support custom commit numbering"); // Two phase to ensure we write metadata in single Write operation using MemoryStream ms = new(); using (BinaryWriter writer = new(ms)) diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index a52ac337e..55895560c 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -9,7 +9,9 @@ using System.Threading; using System.Threading.Tasks; using FASTER.core; +using NUnit.Compatibility; using NUnit.Framework; +using NUnit.Framework.Internal; namespace FASTER.test { @@ -60,6 +62,8 @@ internal class FasterLogTestBase protected static readonly byte[] entry = new byte[100]; protected static readonly ReadOnlySpanBatch spanBatch = new ReadOnlySpanBatch(10000); + private bool deleteOnClose; + protected struct ReadOnlySpanBatch : IReadOnlySpanBatch { private readonly int batchSize; @@ -68,20 +72,23 @@ protected struct ReadOnlySpanBatch : IReadOnlySpanBatch public int TotalEntries() => batchSize; } - protected void BaseSetup() + protected void BaseSetup(bool overwriteCommit = true, bool deleteOnClose = true) { path = TestUtils.MethodTestDir + "/"; // Clean up log files from previous test runs in case they weren't cleaned up TestUtils.DeleteDirectory(path, wait: true); - manager = new DeviceLogCommitCheckpointManager(new LocalStorageNamedDeviceFactory(deleteOnClose: true), new DefaultCheckpointNamingScheme(path)); + manager = new DeviceLogCommitCheckpointManager(new LocalStorageNamedDeviceFactory(deleteOnClose: deleteOnClose), new DefaultCheckpointNamingScheme(path), overwriteLogCommits: overwriteCommit); + this.deleteOnClose = deleteOnClose; } protected void BaseTearDown() { log?.Dispose(); log = null; + if (!deleteOnClose) + manager.PurgeAll(); manager?.Dispose(); manager = null; device?.Dispose(); @@ -783,6 +790,160 @@ public async ValueTask RefreshUncommittedAsyncTest([Values] IteratorType iterato } log.Dispose(); } + } + + [TestFixture] + internal class FasterLogCustomCommitTests : FasterLogTestBase + { + [SetUp] + public void Setup() => base.BaseSetup(false, false); + + [TearDown] + public void TearDown() => base.BaseTearDown(); + + [Test] + [Category("FasterLog")] + [Category("Smoke")] + public void FasterLogSimpleCommitCookieTest([Values] bool autoCommit) + { + var cookie = new byte[100]; + new Random().NextBytes(cookie); + + device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = autoCommit}; + log = new FasterLog(logSettings); + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + Assert.IsTrue(log.AddCommitCookie(cookie)); + log.Commit(true); + + var recoveredLog = new FasterLog(logSettings); + Assert.AreEqual(cookie, recoveredLog.RecoveredCookie); + recoveredLog.Dispose(); + } + + [Test] + [Category("FasterLog")] + public void FasterLogManualCommitTest() + { + device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = false}; + log = new FasterLog(logSettings); + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var commit1Addr = log.TailAddress; + var cookie1 = new byte[100]; + new Random().NextBytes(cookie1); + log.AddCommitCookie(cookie1); + log.CommitManuallyAtNum(1, true); + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var commit2Addr = log.TailAddress; + var cookie2 = new byte[100]; + new Random().NextBytes(cookie2); + log.AddCommitCookie(cookie2); + log.CommitManuallyAtNum(2, true); + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var commit6Addr = log.TailAddress; + var cookie6 = new byte[100]; + new Random().NextBytes(cookie6); + log.AddCommitCookie(cookie6); + log.CommitManuallyAtNum(6, true); + + var recoveredLog = new FasterLog(logSettings, 1); + Assert.AreEqual(cookie1, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit1Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + + recoveredLog = new FasterLog(logSettings, 2); + Assert.AreEqual(cookie2, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit2Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + + // Default argument should recover to most recent + recoveredLog = new FasterLog(logSettings); + Assert.AreEqual(cookie6, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit6Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + } + + [Test] + [Category("FasterLog")] + public void FasterLogMultiCommitCookieTest() + { + var cookies = new List(); + var cookieStatus = new List<(long, bool)>(); + for (var i = 0; i < 10; i++) + { + var cookie = new byte[100]; + new Random().NextBytes(cookie); + cookies.Add(cookie); + } + device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = true}; + log = new FasterLog(logSettings); + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (var j = 0; j < 10; j++) + { + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var success = log.AddCommitCookie(cookies[j]); + cookieStatus.Add(ValueTuple.Create(log.TailAddress, success)); + } + + foreach (var commitNum in manager.ListCommits()) + { + var recoveredLog = new FasterLog(logSettings, commitNum); + var recoveredTail = recoveredLog.TailAddress; + + // Check that the closest successful cookie is recovered + byte[] expectedCookie = null; + for (var i = 0; i < 10; i++) + { + if (!cookieStatus[i].Item2) continue; + if (cookieStatus[i].Item1 > recoveredTail) break; + expectedCookie = cookies[i]; + } + + Assert.AreEqual(expectedCookie, recoveredLog.RecoveredCookie); + recoveredLog.Dispose(); + } + + } + + } } From 32cf7ea5bb3ace2c0320e90e24cc93783d0bf4ce Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 21 Oct 2021 17:32:09 +0000 Subject: [PATCH 09/33] Add management utility for LogCommitManager --- .../DeviceLogCommitCheckpointManager.cs | 13 +++++++++++++ cs/src/core/Index/FasterLog/ILogCommitManager.cs | 11 +++++++++++ .../Index/FasterLog/LocalLogCommitManager.cs | 16 ++++++++++++++++ cs/test/FasterLogTests.cs | 4 +--- 4 files changed, 41 insertions(+), 3 deletions(-) diff --git a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs index 16f06181b..a82cdf2dd 100644 --- a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs +++ b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs @@ -136,6 +136,19 @@ public IEnumerable ListCommits() return deviceFactory.ListContents(checkpointNamingScheme.FasterLogCommitBasePath()).Select(e => checkpointNamingScheme.CommitNumber(e)).OrderByDescending(e => e); } + /// + public void RemoveCommit(long commitNum) + { + deviceFactory.Delete(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); + } + + /// + public void RemoveAllCommits() + { + foreach (var commitNum in ListCommits()) + RemoveCommit(commitNum); + } + /// public byte[] GetCommitMetadata(long commitNum) { diff --git a/cs/src/core/Index/FasterLog/ILogCommitManager.cs b/cs/src/core/Index/FasterLog/ILogCommitManager.cs index 015ae1468..5d65ba8ec 100644 --- a/cs/src/core/Index/FasterLog/ILogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/ILogCommitManager.cs @@ -34,5 +34,16 @@ public interface ILogCommitManager : IDisposable /// /// public IEnumerable ListCommits(); + + /// + /// Remove the given commit, if present + /// + /// commit num to remove + public void RemoveCommit(long commitNum); + + /// + /// Remove all log commits from this manager + /// + public void RemoveAllCommits(); } } \ No newline at end of file diff --git a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs index e803a4898..12775c9a5 100644 --- a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs @@ -78,5 +78,21 @@ public IEnumerable ListCommits() // we only use a single commit file in this implementation yield return 0; } + + /// + public void RemoveCommit(long commitNum) + { + // we only use a single commit file in this implementation + if (commitNum != 0) return; + File.Delete(commitFile); + } + + /// + public void RemoveAllCommits() + { + // we only use a single commit file in this implementation + RemoveCommit(0); + } + } } \ No newline at end of file diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 55895560c..4bd03ceb7 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -88,7 +88,7 @@ protected void BaseTearDown() log?.Dispose(); log = null; if (!deleteOnClose) - manager.PurgeAll(); + manager.RemoveAllCommits(); manager?.Dispose(); manager = null; device?.Dispose(); @@ -943,7 +943,5 @@ public void FasterLogMultiCommitCookieTest() } } - - } } From 617f0cfcdee071655b22c3ca4715da21c8b4a151 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sat, 23 Oct 2021 16:24:37 +0000 Subject: [PATCH 10/33] Handle race on manual commit code path where not the latest tail is written out --- cs/src/core/Index/FasterLog/FasterLog.cs | 84 +++++++++++++++++------- 1 file changed, 61 insertions(+), 23 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index d1314276b..eb866eaf4 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -20,21 +20,39 @@ namespace FASTER.core internal struct ManualCheckpointRequest { private long proposedCommitNum; + private long untilOffset; - internal bool IsSet() => proposedCommitNum != 0; - - internal long DrainCommitNum() + internal bool ShouldCommitManually(CommitInfo commitInfo) { - var result = Interlocked.Exchange(ref proposedCommitNum, 0); - if (result == 0) return -1; - return result; + // This means that there are no current set requests + if (proposedCommitNum == 0) return false; + // If offset is 0, commit request is still trying to set offsets, spin wait while that happens. Will never + // trigger when auto committing + while (proposedCommitNum != 0 && untilOffset == 0) {} + // Otherwise, check that the commit content equals the ones requested. offset field will not be updated + // concurrently as it is protected by proposedCommitNum not being set back to 0 + if (proposedCommitNum == 0) return false; + return commitInfo.UntilAddress == untilOffset; } - internal bool TrySetCommitNum(long num) + internal bool ReserveCommitNum(long num) { return Interlocked.CompareExchange(ref proposedCommitNum, num, 0) == 0; } - + + internal void FinishCommitRequest(long untilOffset) + { + this.untilOffset = untilOffset; + } + + internal long DrainCommitNum() + { + // set offset to 0 first so any successful TrySetCommitNum has a chance to set their own offset after + // cmpxchg succeeds + untilOffset = 0; + var result = Interlocked.Exchange(ref proposedCommitNum, 0); + return result; + } } @@ -595,18 +613,15 @@ public async ValueTask> CommitAsync(Task /// Issue commit request for log (until tail), persisting as the given commitNum. It is up to the users to ensure - /// that the supplied commitNum is unique and monotonically increasing. + /// that the supplied commitNum is unique and monotonically increasing to ensure correctness here. /// /// the commit num to write out /// spin until commit is complete - /// whether manual request is successfully triggered. If not, request should be retried later. - public bool CommitManuallyAtNum(long commitNum, bool spinWait = false) + public void CommitManuallyAtNum(long commitNum, bool spinWait = false) { if (autoCommitOnFlush) throw new FasterException("Specifying of commit num is only supported when auto commit is turned off"); - if (!currentManualRequest.TrySetCommitNum(commitNum)) return false; - CommitInternal(spinWait); - return true; + CommitInternal(spinWait, commitNum); } /// @@ -1018,6 +1033,9 @@ private void CommitCallback(CommitInfo commitInfo) private void SerialCommitCallbackWorker(CommitInfo commitInfo) { + // AutoCommit is off and there is no outstanding request --- safe to skip + if (!autoCommitOnFlush && !currentManualRequest.ShouldCommitManually(commitInfo)) return; + byte[] cookie = null; // Not expected to have heavy contention if (commitCookies != null) @@ -1040,9 +1058,6 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) } } - // AutoCommit is off and there is no outstanding request --- safe to skip - if (!autoCommitOnFlush && !currentManualRequest.IsSet()) return; - // Check if commit is already covered if (CommittedBeginAddress >= BeginAddress && CommittedUntilAddress >= commitInfo.UntilAddress && @@ -1074,7 +1089,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) // Take snapshot of persisted iterators info.SnapshotIterators(PersistedIterators); - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), currentManualRequest.DrainCommitNum()); + logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), autoCommitOnFlush ? -1 : currentManualRequest.DrainCommitNum()); LastPersistedIterators = info.Iterators; CommittedBeginAddress = info.BeginAddress; @@ -1480,13 +1495,24 @@ private long CommitInternal(bool spinWait = false, long commitNum = -1) if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); + if (!autoCommitOnFlush) + { + // when manually committing, ensure that we are serializing commit requests. + // we expect manual commit mode to mostly have sequential commit requests so this is fine + while (true) + { + if (currentManualRequest.ReserveCommitNum(commitNum)) break; + Thread.Yield(); + } + } + epoch.Resume(); - if (autoCommitOnFlush == false) - currentManualRequest.TrySetCommitNum(commitNum); - if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) { + if (!autoCommitOnFlush) + currentManualRequest.FinishCommitRequest(tailAddress); + if (spinWait) { while (CommittedUntilAddress < tailAddress) @@ -1502,20 +1528,32 @@ private long CommitInternal(bool spinWait = false, long commitNum = -1) { // May need to commit begin address and/or iterators epoch.Suspend(); + var beginAddress = allocator.BeginAddress; // Reading cookieChanged concurrently without locking is fine here --- if interleaved with a concurrent // AddCommitCookie call, the function will not properly trigger a checkpoint, and users will perceive // it as if AddCommitCookie is ordered after the Commit call. if (beginAddress > CommittedBeginAddress || IteratorsChanged() || cookieChanged) { + var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + Interlocked.Increment(ref commitMetadataVersion); + // Here, even though a concurrent auto flush commit may have the same until address, it would write + // out the same content as this manual commit, so there is no need to distinguish + if (!autoCommitOnFlush) + currentManualRequest.FinishCommitRequest(untilAddress); + CommitCallback(new CommitInfo { - FromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, - UntilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress, + FromAddress = fromAddress, + UntilAddress = untilAddress, ErrorCode = 0, }); } + // Otherwise, release manual commit request if necessary as no request will be written out + else if (!autoCommitOnFlush) + currentManualRequest.DrainCommitNum(); } return tailAddress; From cc5b8620622ab02ac17110352bc758fb04d51351 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 24 Oct 2021 19:27:34 +0000 Subject: [PATCH 11/33] Rough sketch of new commit code path --- cs/src/core/Index/FasterLog/FasterLog.cs | 282 ++++++++++-------- .../Index/FasterLog/FasterLogRecoveryInfo.cs | 2 + .../core/Index/FasterLog/FasterLogSettings.cs | 8 +- cs/test/FasterLogTests.cs | 8 +- 4 files changed, 161 insertions(+), 139 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index eb866eaf4..fe65b0574 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -16,46 +16,6 @@ namespace FASTER.core { - // Manual checkpoint request with information about the desired commit num - internal struct ManualCheckpointRequest - { - private long proposedCommitNum; - private long untilOffset; - - internal bool ShouldCommitManually(CommitInfo commitInfo) - { - // This means that there are no current set requests - if (proposedCommitNum == 0) return false; - // If offset is 0, commit request is still trying to set offsets, spin wait while that happens. Will never - // trigger when auto committing - while (proposedCommitNum != 0 && untilOffset == 0) {} - // Otherwise, check that the commit content equals the ones requested. offset field will not be updated - // concurrently as it is protected by proposedCommitNum not being set back to 0 - if (proposedCommitNum == 0) return false; - return commitInfo.UntilAddress == untilOffset; - } - - internal bool ReserveCommitNum(long num) - { - return Interlocked.CompareExchange(ref proposedCommitNum, num, 0) == 0; - } - - internal void FinishCommitRequest(long untilOffset) - { - this.untilOffset = untilOffset; - } - - internal long DrainCommitNum() - { - // set offset to 0 first so any successful TrySetCommitNum has a chance to set their own offset after - // cmpxchg succeeds - untilOffset = 0; - var result = Interlocked.Exchange(ref proposedCommitNum, 0); - return result; - } - } - - /// /// FASTER log /// @@ -71,20 +31,20 @@ public class FasterLog : IDisposable private readonly WorkQueueLIFO commitQueue; internal readonly bool readOnlyMode; + internal readonly bool fastCommitMode; private TaskCompletionSource commitTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - - // Manual vs. Auto commit mode - private readonly bool autoCommitOnFlush; - private ManualCheckpointRequest currentManualRequest; // Commit Cookie Related private Queue<(long, byte[])> commitCookies; private long cookieUntilAddr; private bool cookieChanged; + + // Offsets for all currently unprocessed commit records + private Queue<(long, FasterLogRecoveryInfo)> outstandingCommitRecords; /// /// Beginning address of log @@ -223,9 +183,10 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) allocator.HeadAddress = long.MaxValue; } - autoCommitOnFlush = logSettings.AutoCommitOnFlush; + fastCommitMode = logSettings.FastCommitMode; + cookieChanged = false; - currentManualRequest = new ManualCheckpointRequest(); + outstandingCommitRecords = new Queue<(long, FasterLogRecoveryInfo)>(); } /// @@ -619,8 +580,6 @@ public async ValueTask> CommitAsync(Taskspin until commit is complete public void CommitManuallyAtNum(long commitNum, bool spinWait = false) { - if (autoCommitOnFlush) - throw new FasterException("Specifying of commit num is only supported when auto commit is turned off"); CommitInternal(spinWait, commitNum); } @@ -1028,14 +987,49 @@ private int Align(int length) /// private void CommitCallback(CommitInfo commitInfo) { + // Using count is safe as a fast filtering mechanism to reduce number of invocations despite concurrency + if (outstandingCommitRecords.Count == 0) return; commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } - private void SerialCommitCallbackWorker(CommitInfo commitInfo) + private void ComputeRecoveryInfo(long commitTail, ref FasterLogRecoveryInfo recoveryInfo) { - // AutoCommit is off and there is no outstanding request --- safe to skip - if (!autoCommitOnFlush && !currentManualRequest.ShouldCommitManually(commitInfo)) return; + recoveryInfo.BeginAddress = BeginAddress; + recoveryInfo.FlushedUntilAddress = commitTail; + recoveryInfo.Cookie = FindCommitCookie(commitTail); + recoveryInfo.SnapshotIterators(PersistedIterators); + } + + private unsafe bool TryEnqueueCommitRecord(long commitTail, ref FasterLogRecoveryInfo info, out long logicalAddress) + { + logicalAddress = 0; + + ComputeRecoveryInfo(commitTail, ref info); + var entryBody = info.ToByteArray(); + var length = entryBody.Length; + int allocatedLength = headerSize + Align(length); + ValidateAllocatedLength(allocatedLength); + + epoch.Resume(); + + logicalAddress = allocator.TryAllocateRetryNow(length); + if (logicalAddress == 0) + { + epoch.Suspend(); + return false; + } + + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); + fixed (byte* bp = entryBody) + Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); + SetCommitRecordHeader(length, (byte*)physicalAddress); + epoch.Suspend(); + return true; + } + + private byte[] FindCommitCookie(long commitUntilAddress) + { byte[] cookie = null; // Not expected to have heavy contention if (commitCookies != null) @@ -1047,7 +1041,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) while (true) { var (addr, bytes) = commitCookies.Peek(); - if (addr > commitInfo.UntilAddress) break; + if (addr > commitUntilAddress) break; cookie = bytes; if (commitCookies.Count == 1) break; commitCookies.Dequeue(); @@ -1057,49 +1051,71 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) } } } + return cookie; + } - // Check if commit is already covered - if (CommittedBeginAddress >= BeginAddress && - CommittedUntilAddress >= commitInfo.UntilAddress && - persistedCommitMetadataVersion >= commitMetadataVersion && - commitInfo.ErrorCode == 0) + private void CommitMetadataOnly(ref FasterLogRecoveryInfo info) + { + var beginAddress = allocator.BeginAddress; + // Reading cookieChanged concurrently without locking is fine here --- if interleaved with a concurrent + // AddCommitCookie call, the function will not properly trigger a checkpoint, and users will perceive + // it as if AddCommitCookie is ordered after the Commit call. + if (beginAddress > CommittedBeginAddress || IteratorsChanged(info) || cookieChanged) { - // Should only be a scenario where auto commit is true - Debug.Assert(autoCommitOnFlush); - return; + var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + + Interlocked.Increment(ref commitMetadataVersion); + + CommitCallback(new CommitInfo + { + FromAddress = fromAddress, + UntilAddress = untilAddress, + ErrorCode = 0, + }); } + } + + private void UpdateCommittedState(FasterLogRecoveryInfo recoveryInfo) + { + LastPersistedIterators = recoveryInfo.Iterators; + CommittedBeginAddress = recoveryInfo.BeginAddress; + CommittedUntilAddress = recoveryInfo.FlushedUntilAddress; + recoveryInfo.CommitIterators(PersistedIterators); + // TODO(Tianyu): WTF is going on with metadata version? + } + private void SerialCommitCallbackWorker(CommitInfo commitInfo) + { if (commitInfo.ErrorCode == 0) { - // Capture CMV first, so metadata prior to CMV update is visible to commit - long _localCMV = commitMetadataVersion; - - if (CommittedUntilAddress > commitInfo.FromAddress) - commitInfo.FromAddress = CommittedUntilAddress; - if (CommittedUntilAddress > commitInfo.UntilAddress) - commitInfo.UntilAddress = CommittedUntilAddress; - - FasterLogRecoveryInfo info = new FasterLogRecoveryInfo + var coveredCommits = new List(); + // Check for the commit records included in this flush + lock (outstandingCommitRecords) { - BeginAddress = BeginAddress, - FlushedUntilAddress = commitInfo.UntilAddress, - Cookie = cookie - }; - - // Take snapshot of persisted iterators - info.SnapshotIterators(PersistedIterators); - - logCommitManager.Commit(info.BeginAddress, info.FlushedUntilAddress, info.ToByteArray(), autoCommitOnFlush ? -1 : currentManualRequest.DrainCommitNum()); - - LastPersistedIterators = info.Iterators; - CommittedBeginAddress = info.BeginAddress; - CommittedUntilAddress = info.FlushedUntilAddress; - if (_localCMV > persistedCommitMetadataVersion) - persistedCommitMetadataVersion = _localCMV; + while (outstandingCommitRecords.Count != 0) + { + var (addr, recoveryInfo) = outstandingCommitRecords.Peek(); + if (addr >= commitInfo.UntilAddress) break; + coveredCommits.Add(recoveryInfo); + outstandingCommitRecords.Dequeue(); + } + } + if (fastCommitMode) + { + // In fast commit mode, can safely set committed state to the latest flushed + UpdateCommittedState(coveredCommits[coveredCommits.Count - 1]); + } - // Update completed address for persisted iterators - info.CommitIterators(PersistedIterators); + foreach (var recoveryInfo in coveredCommits) + { + // TODO(Tianyu): WTF is up with commit metadata version + logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.FlushedUntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); + // Otherwise, set committed state as we commit metadata explicitly + if (!fastCommitMode) + UpdateCommittedState(recoveryInfo); + } } var _commitTcs = commitTcs; @@ -1116,22 +1132,20 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) _commitTcs.TrySetException(new CommitFailureException(lci, $"Commit of address range [{commitInfo.FromAddress}-{commitInfo.UntilAddress}] failed with error code {commitInfo.ErrorCode}")); } - private bool IteratorsChanged() + private bool IteratorsChanged(FasterLogRecoveryInfo info) { var _lastPersistedIterators = LastPersistedIterators; if (_lastPersistedIterators == null) { - if (PersistedIterators.Count == 0) - return false; - return true; + return info.Iterators.Count != 0; } - if (_lastPersistedIterators.Count != PersistedIterators.Count) + if (_lastPersistedIterators.Count != info.Iterators.Count) return true; foreach (var item in _lastPersistedIterators) { - if (PersistedIterators.TryGetValue(item.Key, out var other)) + if (info.Iterators.TryGetValue(item.Key, out var other)) { - if (item.Value != other.requestedCompletedUntilAddress) return true; + if (item.Value != other) return true; } else return true; @@ -1495,24 +1509,39 @@ private long CommitInternal(bool spinWait = false, long commitNum = -1) if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); - if (!autoCommitOnFlush) + // set the content of this commit to the current tail and base all commit metadata on this address, even + // though perhaps more entries will be flushed as part of this commit + long commitTail, commitRecordOffset; + var info = new FasterLogRecoveryInfo{ CommitNum = commitNum }; + + // Protect the creation of commit record and enqueuing into outstandingCommitRecords under a critical section, + // such that the long address are sorted in outstandingCommitRecords. Ok because we do not expect heavy + // contention on the commit code path + lock (outstandingCommitRecords) { - // when manually committing, ensure that we are serializing commit requests. - // we expect manual commit mode to mostly have sequential commit requests so this is fine - while (true) + do { - if (currentManualRequest.ReserveCommitNum(commitNum)) break; - Thread.Yield(); - } + commitTail = TailAddress; + } while (!TryEnqueueCommitRecord(commitTail, ref info, out commitRecordOffset)); + + // Enqueue the commit record's content and offset into the queue so it can be picked up by the next flush + outstandingCommitRecords.Enqueue(ValueTuple.Create(commitRecordOffset, info)); } + // Need to check, however, that a concurrent flush hasn't already advanced flushed address past this + // commit. If so, need to manually trigger another commit callback in case the earlier flush missed our + // commit record + if (commitRecordOffset < FlushedUntilAddress) + { + CommitMetadataOnly(ref info); + return commitTail; + } + + // Otherwise, move to set read-only tail and flush epoch.Resume(); - if (allocator.ShiftReadOnlyToTail(out long tailAddress, out _)) + if (allocator.ShiftReadOnlyToTail(out var tailAddress, out _)) { - if (!autoCommitOnFlush) - currentManualRequest.FinishCommitRequest(tailAddress); - if (spinWait) { while (CommittedUntilAddress < tailAddress) @@ -1528,35 +1557,10 @@ private long CommitInternal(bool spinWait = false, long commitNum = -1) { // May need to commit begin address and/or iterators epoch.Suspend(); - - var beginAddress = allocator.BeginAddress; - // Reading cookieChanged concurrently without locking is fine here --- if interleaved with a concurrent - // AddCommitCookie call, the function will not properly trigger a checkpoint, and users will perceive - // it as if AddCommitCookie is ordered after the Commit call. - if (beginAddress > CommittedBeginAddress || IteratorsChanged() || cookieChanged) - { - var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; - var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; - - Interlocked.Increment(ref commitMetadataVersion); - // Here, even though a concurrent auto flush commit may have the same until address, it would write - // out the same content as this manual commit, so there is no need to distinguish - if (!autoCommitOnFlush) - currentManualRequest.FinishCommitRequest(untilAddress); - - CommitCallback(new CommitInfo - { - FromAddress = fromAddress, - UntilAddress = untilAddress, - ErrorCode = 0, - }); - } - // Otherwise, release manual commit request if necessary as no request will be written out - else if (!autoCommitOnFlush) - currentManualRequest.DrainCommitNum(); + CommitMetadataOnly(ref info); } - return tailAddress; + return commitTail; } private long RefreshUncommittedInternal(bool spinWait = false) @@ -1625,6 +1629,22 @@ private unsafe void SetHeader(int length, byte* dest) *(ulong*)dest = Utility.XorBytes(dest + 8, length + 4); } } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private unsafe void SetCommitRecordHeader(int length, byte* dest) + { + // commit record has negative length field to differentiate from normal records + if (logChecksum == LogChecksumType.None) + { + *(int*)dest = -length; + return; + } + else if (logChecksum == LogChecksumType.PerEntry) + { + *(int*)(dest + 8) = -length; + *(ulong*)dest = Utility.XorBytes(dest + 8, length + 4); + } + } [MethodImpl(MethodImplOptions.AggressiveInlining)] private void ValidateAllocatedLength(int numSlots) diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index 0a880b2a8..c5a989232 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -35,6 +35,8 @@ internal struct FasterLogRecoveryInfo /// User-specified commit cookie /// public byte[] Cookie; + + public long CommitNum; /// /// Initialize diff --git a/cs/src/core/Index/FasterLog/FasterLogSettings.cs b/cs/src/core/Index/FasterLog/FasterLogSettings.cs index c45107346..ce1e290eb 100644 --- a/cs/src/core/Index/FasterLog/FasterLogSettings.cs +++ b/cs/src/core/Index/FasterLog/FasterLogSettings.cs @@ -90,11 +90,11 @@ public class FasterLogSettings public bool ReadOnlyMode = false; /// - /// When set to true --- automatically create a commit for every disk flush (which may happen in the background - /// due to enqueue operations). Otherwise, a commit will only be created and recoverable where Commit() is \ - /// invoked manually + /// When FastCommitMode is enabled, FasterLog will reduce commit critical path latency, but may result in slower + /// recovery to a commit on restart. Additionally, FastCommitMode is only possible when log checksum is turned + /// on. /// - public bool AutoCommitOnFlush = true; + public bool FastCommitMode = false; internal LogSettings GetLogSettings() { diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 4bd03ceb7..56b99f473 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -804,13 +804,13 @@ internal class FasterLogCustomCommitTests : FasterLogTestBase [Test] [Category("FasterLog")] [Category("Smoke")] - public void FasterLogSimpleCommitCookieTest([Values] bool autoCommit) + public void FasterLogSimpleCommitCookieTest() { var cookie = new byte[100]; new Random().NextBytes(cookie); device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = autoCommit}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; @@ -835,7 +835,7 @@ public void FasterLogSimpleCommitCookieTest([Values] bool autoCommit) public void FasterLogManualCommitTest() { device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = false}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; @@ -906,7 +906,7 @@ public void FasterLogMultiCommitCookieTest() } device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager, AutoCommitOnFlush = true}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; From a987f0f63de392adc46556de66dc2019ca4dab85 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Mon, 25 Oct 2021 20:47:26 +0000 Subject: [PATCH 12/33] save work --- cs/src/core/Index/FasterLog/FasterLog.cs | 269 +++++++----------- .../core/Index/FasterLog/FasterLogIterator.cs | 79 ++++- .../Index/FasterLog/FasterLogRecoveryInfo.cs | 29 +- .../core/Index/FasterLog/ILogCommitManager.cs | 2 +- .../Index/FasterLog/LocalLogCommitManager.cs | 5 +- cs/test/FasterLogTests.cs | 67 +---- 6 files changed, 195 insertions(+), 256 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index fe65b0574..ebd64e976 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -37,14 +37,10 @@ private TaskCompletionSource commitTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - - // Commit Cookie Related - private Queue<(long, byte[])> commitCookies; - private long cookieUntilAddr; - private bool cookieChanged; - + // Offsets for all currently unprocessed commit records private Queue<(long, FasterLogRecoveryInfo)> outstandingCommitRecords; + private long commitNum; /// /// Beginning address of log @@ -115,7 +111,7 @@ internal readonly ConcurrentDictionary PersistedI /// /// Committed view of commitMetadataVersion /// - private long persistedCommitMetadataVersion; + private long persistedCommitNum; internal Dictionary LastPersistedIterators; @@ -185,7 +181,6 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) fastCommitMode = logSettings.FastCommitMode; - cookieChanged = false; outstandingCommitRecords = new Queue<(long, FasterLogRecoveryInfo)>(); } @@ -445,7 +440,7 @@ public void WaitForCommit(long untilAddress = 0) var tailAddress = untilAddress; if (tailAddress == 0) tailAddress = allocator.GetTailAddress(); - while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) Thread.Yield(); + while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) Thread.Yield(); } /// @@ -463,10 +458,10 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok var tailAddress = untilAddress; if (tailAddress == 0) tailAddress = allocator.GetTailAddress(); - while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) { var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) task = linkedCommitInfo.NextTask; else break; @@ -476,54 +471,14 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok #region Commit and CommitAsync - /// - /// Associates the given cookie (an opaque chunk of client-specified bytes) with a log address (or -1 for - /// current tail). If the method returns true, the cookie will be recovered in RecoveredCookie as part of any - /// commit including the log address, unless overwritten by a later cookie. - /// - /// Thread-safe. Generally, cookies associated with larger address are considered more recent. Behavior is - /// arbitrary if multiple cookies are associated with the same address concurrently. - /// - /// custom byte chunks to persist with commits - /// address to associate cookie with, or -1 for current tail - /// whether the cookie will be present with all future commits including the lower bound - public bool AddCommitCookie(byte[] cookie, long addrLowerBound = -1) - { - if (addrLowerBound == -1) addrLowerBound = TailAddress; - - // Lazily allocate data structure - if (commitCookies == null) - Interlocked.CompareExchange(ref commitCookies, new Queue<(long, byte[])>(), null); - - // Not expected to have heavy contention - lock (commitCookies) - { - // Ensure that cookie is going to be a part of any commit that includes the specified addrLowerBound - if (CommittedUntilAddress >= addrLowerBound) return false; - - // Reject any out-of-order commit cookies as they will never be written out - if (cookieUntilAddr > addrLowerBound) return false; - - // Because SerialCommitCallbackWorker will also access the queue under mutex, it is safe to assume - // anything that is enqueued here will be written out later - cookieUntilAddr = addrLowerBound; - commitCookies.Enqueue(ValueTuple.Create(addrLowerBound, cookie)); - - // If associating a new cookie with the same address, mark so commit code path can pick up the change - if (cookieUntilAddr == addrLowerBound) - cookieChanged = true; - } - return true; - } - /// /// Issue commit request for log (until tail) /// /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. /// - public void Commit(bool spinWait = false) + public long Commit(bool spinWait = false, byte[] cookie = null, long commitNum = -1) { - CommitInternal(spinWait); + return CommitInternal(spinWait, cookie, commitNum); } /// @@ -532,16 +487,16 @@ public void Commit(bool spinWait = false) /// ongoing commit fails. /// /// - public async ValueTask CommitAsync(CancellationToken token = default) + public async ValueTask CommitAsync(CancellationToken token = default, byte[] cookie = null, long commitNum = -1) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(); + var tailAddress = CommitInternal(false, cookie, commitNum); - while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) { var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) task = linkedCommitInfo.NextTask; else break; @@ -554,16 +509,16 @@ public async ValueTask CommitAsync(CancellationToken token = default) /// from prevCommitTask to current fails. /// /// - public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default) + public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default, byte[] cookie = null, long commitNum = -1) { token.ThrowIfCancellationRequested(); if (prevCommitTask == null) prevCommitTask = CommitTask; - var tailAddress = CommitInternal(); + var tailAddress = CommitInternal(false, cookie, commitNum); - while (CommittedUntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) { var linkedCommitInfo = await prevCommitTask.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitMetadataVersion < commitMetadataVersion) + if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) prevCommitTask = linkedCommitInfo.NextTask; else return linkedCommitInfo.NextTask; @@ -572,17 +527,6 @@ public async ValueTask> CommitAsync(Task - /// Issue commit request for log (until tail), persisting as the given commitNum. It is up to the users to ensure - /// that the supplied commitNum is unique and monotonically increasing to ensure correctness here. - /// - /// the commit num to write out - /// spin until commit is complete - public void CommitManuallyAtNum(long commitNum, bool spinWait = false) - { - CommitInternal(spinWait, commitNum); - } - /// /// Trigger a refresh of information about uncommitted part of log (tail address) to ensure visibility /// to uncommitted scan iterators. Will cause SafeTailAddress to reflect the current tail address. @@ -992,87 +936,59 @@ private void CommitCallback(CommitInfo commitInfo) commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } - private void ComputeRecoveryInfo(long commitTail, ref FasterLogRecoveryInfo recoveryInfo) - { - recoveryInfo.BeginAddress = BeginAddress; - recoveryInfo.FlushedUntilAddress = commitTail; - recoveryInfo.Cookie = FindCommitCookie(commitTail); - recoveryInfo.SnapshotIterators(PersistedIterators); - } - - private unsafe bool TryEnqueueCommitRecord(long commitTail, ref FasterLogRecoveryInfo info, out long logicalAddress) + private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info, out long logicalAddress) { logicalAddress = 0; - ComputeRecoveryInfo(commitTail, ref info); - var entryBody = info.ToByteArray(); + var entryBodySize = info.SerializedSize(); - var length = entryBody.Length; - int allocatedLength = headerSize + Align(length); + int allocatedLength = headerSize + Align(entryBodySize); ValidateAllocatedLength(allocatedLength); epoch.Resume(); - logicalAddress = allocator.TryAllocateRetryNow(length); + logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); if (logicalAddress == 0) { epoch.Suspend(); return false; } + // Finish filling in all fields + info.BeginAddress = BeginAddress; + info.UntilAddress = logicalAddress + allocatedLength; var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); + + var entryBody = info.ToByteArray(); fixed (byte* bp = entryBody) - Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); - SetCommitRecordHeader(length, (byte*)physicalAddress); + Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), entryBody.Length, entryBody.Length); + SetCommitRecordHeader(entryBody.Length, (byte*)physicalAddress); epoch.Suspend(); return true; } - private byte[] FindCommitCookie(long commitUntilAddress) + private bool ShouldCommmitMetadata(ref FasterLogRecoveryInfo info) { - byte[] cookie = null; - // Not expected to have heavy contention - if (commitCookies != null) - { - lock (commitCookies) - { - if (commitCookies.Count != 0) - { - while (true) - { - var (addr, bytes) = commitCookies.Peek(); - if (addr > commitUntilAddress) break; - cookie = bytes; - if (commitCookies.Count == 1) break; - commitCookies.Dequeue(); - } - - cookieChanged = false; - } - } - } - return cookie; + return allocator.BeginAddress > CommittedBeginAddress || IteratorsChanged(ref info) || info.Cookie != null; } - - private void CommitMetadataOnly(ref FasterLogRecoveryInfo info) + + private void CommitMetadataOnly(ref FasterLogRecoveryInfo info, bool spinWait) { var beginAddress = allocator.BeginAddress; - // Reading cookieChanged concurrently without locking is fine here --- if interleaved with a concurrent - // AddCommitCookie call, the function will not properly trigger a checkpoint, and users will perceive - // it as if AddCommitCookie is ordered after the Commit call. - if (beginAddress > CommittedBeginAddress || IteratorsChanged(info) || cookieChanged) + var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + + CommitCallback(new CommitInfo { - var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; - var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + FromAddress = fromAddress, + UntilAddress = untilAddress, + ErrorCode = 0, + }); - Interlocked.Increment(ref commitMetadataVersion); - - CommitCallback(new CommitInfo - { - FromAddress = fromAddress, - UntilAddress = untilAddress, - ErrorCode = 0, - }); + if (spinWait) + { + while (info.CommitNum < persistedCommitNum) + Thread.Yield(); } } @@ -1080,9 +996,9 @@ private void UpdateCommittedState(FasterLogRecoveryInfo recoveryInfo) { LastPersistedIterators = recoveryInfo.Iterators; CommittedBeginAddress = recoveryInfo.BeginAddress; - CommittedUntilAddress = recoveryInfo.FlushedUntilAddress; + CommittedUntilAddress = recoveryInfo.UntilAddress; recoveryInfo.CommitIterators(PersistedIterators); - // TODO(Tianyu): WTF is going on with metadata version? + Utility.MonotonicUpdate(ref persistedCommitNum, recoveryInfo.CommitNum, out _); } private void SerialCommitCallbackWorker(CommitInfo commitInfo) @@ -1110,8 +1026,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) foreach (var recoveryInfo in coveredCommits) { - // TODO(Tianyu): WTF is up with commit metadata version - logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.FlushedUntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); + logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); // Otherwise, set committed state as we commit metadata explicitly if (!fastCommitMode) UpdateCommittedState(recoveryInfo); @@ -1132,7 +1047,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) _commitTcs.TrySetException(new CommitFailureException(lci, $"Commit of address range [{commitInfo.FromAddress}-{commitInfo.UntilAddress}] failed with error code {commitInfo.ErrorCode}")); } - private bool IteratorsChanged(FasterLogRecoveryInfo info) + private bool IteratorsChanged(ref FasterLogRecoveryInfo info) { var _lastPersistedIterators = LastPersistedIterators; if (_lastPersistedIterators == null) @@ -1215,21 +1130,16 @@ private void SignalWaitingROIterators() _commitTcs?.TrySetResult(lci); } - /// - /// Restore log synchronously - /// - private bool Restore(out Dictionary iterators, out byte[] cookie, long requestedVersion = -1) + private bool RestoreLatest(out Dictionary iterators, out byte[] cookie) { iterators = null; cookie = null; - if (requestedVersion != -1) - return RestoreFromCommitNum(requestedVersion, out iterators, out cookie); foreach (var commitNum in logCommitManager.ListCommits()) { try { - return RestoreFromCommitNum(commitNum, out iterators, out cookie); + if (RestoreFromCommitFile(commitNum, out iterators, out cookie)); } catch { } } @@ -1237,7 +1147,15 @@ private bool Restore(out Dictionary iterators, out byte[] cookie, return false; } - private bool RestoreFromCommitNum(long commitNum, out Dictionary iterators, out byte[] cookie) + /// + /// Restore log synchronously + /// + private bool RestoreCommitNum(long commitNum, out Dictionary iterators, out byte[] cookie) + { + throw new NotImplementedException(); + } + + private bool RestoreFromCommitFile(long commitNum, out Dictionary iterators, out byte[] cookie) { iterators = null; cookie = null; @@ -1248,7 +1166,7 @@ private bool RestoreFromCommitNum(long commitNum, out Dictionary i } if (headAddress > 0) - allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.FlushedUntilAddress, info.FlushedUntilAddress); + allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress); iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; @@ -1268,7 +1186,7 @@ private async ValueTask> RestoreAsync(CancellationToken return default; if (headAddress > 0) - await allocator.RestoreHybridLogAsync(info.BeginAddress, headAddress, info.FlushedUntilAddress, info.FlushedUntilAddress, cancellationToken: cancellationToken).ConfigureAwait(false); + await allocator.RestoreHybridLogAsync(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress, cancellationToken: cancellationToken).ConfigureAwait(false); return CompleteRestoreFromCommit(info); } @@ -1296,7 +1214,7 @@ private bool PrepareToRestoreFromCommit(long commitNum, out FasterLogRecoveryInf if (!readOnlyMode) { - headAddress = info.FlushedUntilAddress - allocator.GetOffsetInPage(info.FlushedUntilAddress); + headAddress = info.UntilAddress - allocator.GetOffsetInPage(info.UntilAddress); if (info.BeginAddress > headAddress) headAddress = info.BeginAddress; @@ -1309,9 +1227,9 @@ private bool PrepareToRestoreFromCommit(long commitNum, out FasterLogRecoveryInf private Dictionary CompleteRestoreFromCommit(FasterLogRecoveryInfo info) { - CommittedUntilAddress = info.FlushedUntilAddress; + CommittedUntilAddress = info.UntilAddress; CommittedBeginAddress = info.BeginAddress; - SafeTailAddress = info.FlushedUntilAddress; + SafeTailAddress = info.UntilAddress; // Fix uncommitted addresses in iterators var recoveredIterators = info.Iterators; @@ -1504,63 +1422,74 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private long CommitInternal(bool spinWait = false, long commitNum = -1) + private long CommitInternal(bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) { if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); // set the content of this commit to the current tail and base all commit metadata on this address, even // though perhaps more entries will be flushed as part of this commit - long commitTail, commitRecordOffset; - var info = new FasterLogRecoveryInfo{ CommitNum = commitNum }; + long commitRecordOffset; + var info = new FasterLogRecoveryInfo(); + - // Protect the creation of commit record and enqueuing into outstandingCommitRecords under a critical section, - // such that the long address are sorted in outstandingCommitRecords. Ok because we do not expect heavy - // contention on the commit code path + // This critical section serializes commit record creation and ensures that the long address are sorted in + // outstandingCommitRecords. Ok because we do not expect heavy contention on the commit code path lock (outstandingCommitRecords) { + if (proposedCommitNum == -1) + info.CommitNum = commitNum++; + else if (proposedCommitNum > commitNum) + info.CommitNum = commitNum = proposedCommitNum; + else + // Invalid commit num + return -1; + + + info.Cookie = cookie; do { - commitTail = TailAddress; - } while (!TryEnqueueCommitRecord(commitTail, ref info, out commitRecordOffset)); - + info.SnapshotIterators(PersistedIterators); + if (CommittedUntilAddress == TailAddress && !ShouldCommmitMetadata(ref info)) + { + // Nothing to commit if no metadata update and no new entries, use -1 to denote that + return -1; + } + } while (!TryEnqueueCommitRecord(ref info, out commitRecordOffset)); + // Enqueue the commit record's content and offset into the queue so it can be picked up by the next flush + // At this point, we expect the commit record to be flushed out as a distinct recovery point outstandingCommitRecords.Enqueue(ValueTuple.Create(commitRecordOffset, info)); } // Need to check, however, that a concurrent flush hasn't already advanced flushed address past this - // commit. If so, need to manually trigger another commit callback in case the earlier flush missed our - // commit record + // commit. If so, need to manually trigger another commit callback in case the one triggered by the flush + // already finished execution and missed our commit record if (commitRecordOffset < FlushedUntilAddress) { - CommitMetadataOnly(ref info); - return commitTail; + CommitMetadataOnly(ref info, spinWait); + return info.UntilAddress; } // Otherwise, move to set read-only tail and flush epoch.Resume(); - if (allocator.ShiftReadOnlyToTail(out var tailAddress, out _)) + if (allocator.ShiftReadOnlyToTail(out _, out _)) { if (spinWait) { - while (CommittedUntilAddress < tailAddress) + while (CommittedUntilAddress < commitRecordOffset) { epoch.ProtectAndDrain(); Thread.Yield(); } } - epoch.Suspend(); - - } - else - { - // May need to commit begin address and/or iterators - epoch.Suspend(); - CommitMetadataOnly(ref info); } - - return commitTail; + epoch.Suspend(); + // No need to trigger a manual commit even if no flush is triggered here -- if someone flushed the commit + // record between our last check of FlushedUntilAddress and now, their flush callback would observe + // this commitRecord in outstandingCommitRecords and commit it for us + return info.UntilAddress; } private long RefreshUncommittedInternal(bool spinWait = false) diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 77395d25e..9c06ecbe9 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -5,6 +5,7 @@ using System.Buffers; using System.Collections.Generic; using System.Diagnostics; +using System.IO; using System.Runtime.CompilerServices; using System.Threading; using System.Threading.Tasks; @@ -201,10 +202,14 @@ public unsafe bool GetNext(out byte[] entry, out int entryLength, out long curre nextAddress = default; return false; } - epoch.Resume(); - if (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress)) + + // Continue looping until we find a record that is not a commit record + while (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress, + out var isCommitRecord)) { + if (isCommitRecord) continue; + if (getMemory != null) { // Use user delegate to allocate memory @@ -264,8 +269,12 @@ public unsafe bool GetNext(MemoryPool pool, out IMemoryOwner entry, } epoch.Resume(); - if (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress)) + + // Continue looping until we find a record that is not a commit record + while (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress, out var isCommitRecord)) { + if (isCommitRecord) continue; + entry = pool.Rent(entryLength); fixed (byte* bp = &entry.Memory.Span.GetPinnableReference()) @@ -275,6 +284,7 @@ public unsafe bool GetNext(MemoryPool pool, out IMemoryOwner entry, return true; } + entry = default; entryLength = default; epoch.Suspend(); @@ -368,6 +378,48 @@ private int Align(int length) return (length + 3) & ~3; } + internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info) + { + epoch.Resume(); + var foundCommit = false; + try + { + // Continue looping until we find a record that is a commit record + while (GetNextInternal(out long physicalAddress, out var entryLength, out currentAddress, + out nextAddress, + out var isCommitRecord)) + { + if (!isCommitRecord) continue; + + foundCommit = true; + byte[] entry; + if (getMemory != null) + { + // Use user delegate to allocate memory + entry = getMemory(entryLength); + if (entry.Length < entryLength) + throw new FasterException("Byte array provided has invalid length"); + } + else + { + // We allocate a byte array from heap + entry = new byte[entryLength]; + } + + fixed (byte* bp = entry) + Buffer.MemoryCopy((void*) (headerSize + physicalAddress), bp, entryLength, entryLength); + info.Initialize(new BinaryReader(new MemoryStream(entry))); + } + } + catch (FasterException) + { + // If we are here --- simply stop scanning because we ran into an incomplete entry + } + + epoch.Suspend(); + return foundCommit; + } + /// /// Retrieve physical address of next iterator value /// (under epoch protection if it is from main page buffer) @@ -377,7 +429,7 @@ private int Align(int length) /// /// /// - private unsafe bool GetNextInternal(out long physicalAddress, out int entryLength, out long currentAddress, out long outNextAddress) + private unsafe bool GetNextInternal(out long physicalAddress, out int entryLength, out long currentAddress, out long outNextAddress, out bool commitRecord) { while (true) { @@ -385,6 +437,7 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt entryLength = 0; currentAddress = nextAddress; outNextAddress = nextAddress; + commitRecord = false; // Check for boundary conditions if (currentAddress < allocator.BeginAddress) @@ -441,12 +494,18 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt var curPage = currentAddress >> allocator.LogPageSizeBits; throw new FasterException("Invalid checksum found during scan, skipping page " + curPage); } - else - continue; + continue; + } + + // commit records have negative length fields + if (entryLength < 0) + { + commitRecord = true; + entryLength = -entryLength; } int recordSize = headerSize + Align(entryLength); - if (entryLength < 0 || (_currentOffset + recordSize > allocator.PageSize)) + if (_currentOffset + recordSize > allocator.PageSize) { currentAddress = (1 + (currentAddress >> allocator.LogPageSizeBits)) << allocator.LogPageSizeBits; if (Utility.MonotonicUpdate(ref nextAddress, currentAddress, out _)) @@ -454,8 +513,7 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt epoch.Suspend(); throw new FasterException("Invalid length of record found: " + entryLength + " at address " + currentAddress + ", skipping page"); } - else - continue; + continue; } // Verify checksum if needed @@ -470,8 +528,7 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt epoch.Suspend(); throw new FasterException("Invalid checksum found during scan, skipping page " + curPage); } - else - continue; + continue; } } diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index c5a989232..3032756bd 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -24,7 +24,7 @@ internal struct FasterLogRecoveryInfo /// /// Flushed logical address /// - public long FlushedUntilAddress; + public long UntilAddress; /// /// Persisted iterators @@ -37,17 +37,18 @@ internal struct FasterLogRecoveryInfo public byte[] Cookie; public long CommitNum; - + /// /// Initialize /// public void Initialize() { BeginAddress = 0; - FlushedUntilAddress = 0; + UntilAddress = 0; Iterators = null; Cookie = null; } + /// /// Initialize from stream @@ -62,7 +63,7 @@ public void Initialize(BinaryReader reader) version = reader.ReadInt32(); checkSum = reader.ReadInt64(); BeginAddress = reader.ReadInt64(); - FlushedUntilAddress = reader.ReadInt64(); + UntilAddress = reader.ReadInt64(); } catch (Exception e) { @@ -71,7 +72,7 @@ public void Initialize(BinaryReader reader) if (version != 0) throw new FasterException("Invalid version found during commit recovery"); - if (checkSum != (BeginAddress ^ FlushedUntilAddress)) + if (checkSum != (BeginAddress ^ UntilAddress)) throw new FasterException("Invalid checksum found during commit recovery"); var count = 0; @@ -117,9 +118,9 @@ public readonly byte[] ToByteArray() using (BinaryWriter writer = new(ms)) { writer.Write(0); // version - writer.Write(BeginAddress ^ FlushedUntilAddress); // checksum + writer.Write(BeginAddress ^ UntilAddress); // checksum writer.Write(BeginAddress); - writer.Write(FlushedUntilAddress); + writer.Write(UntilAddress); if (Iterators?.Count > 0) { writer.Write(Iterators.Count); @@ -147,6 +148,18 @@ public readonly byte[] ToByteArray() return ms.ToArray(); } + public int SerializedSize() + { + var iteratorSize = sizeof(int); + if (Iterators != null) + { + foreach (var kvp in Iterators) + iteratorSize += kvp.Key.Length + sizeof(long); + } + + return sizeof(int) + 3 * sizeof(long) + iteratorSize + Cookie?.Length ?? 0; + } + /// /// Take snapshot of persisted iterators /// @@ -187,7 +200,7 @@ public void DebugPrint() Debug.WriteLine("******** Log Commit Info ********"); Debug.WriteLine("BeginAddress: {0}", BeginAddress); - Debug.WriteLine("FlushedUntilAddress: {0}", FlushedUntilAddress); + Debug.WriteLine("FlushedUntilAddress: {0}", UntilAddress); } } } diff --git a/cs/src/core/Index/FasterLog/ILogCommitManager.cs b/cs/src/core/Index/FasterLog/ILogCommitManager.cs index 5d65ba8ec..84fb1acb3 100644 --- a/cs/src/core/Index/FasterLog/ILogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/ILogCommitManager.cs @@ -20,7 +20,7 @@ public interface ILogCommitManager : IDisposable /// Commit metadata - should be persisted /// Proposed commit num, or -1 if none, in which case the implementation will pick one. /// Caller must ensure commit num supplied is unique and monotonically increasing - void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum = -1); + void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum); /// /// Return commit metadata diff --git a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs index 12775c9a5..a22269a57 100644 --- a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs @@ -30,10 +30,9 @@ public LocalLogCommitManager(string commitFile) /// Commit metadata /// Ignored param - public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum = -1) + public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum) { - if (proposedCommitNum != -1) - throw new FasterException("LocalLogCommitManager does not support custom commit numbering"); + // Two phase to ensure we write metadata in single Write operation using MemoryStream ms = new(); using (BinaryWriter writer = new(ms)) diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 56b99f473..8f03eb41c 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -822,8 +822,7 @@ public void FasterLogSimpleCommitCookieTest() log.Enqueue(entry); } - Assert.IsTrue(log.AddCommitCookie(cookie)); - log.Commit(true); + log.Commit(true, cookie); var recoveredLog = new FasterLog(logSettings); Assert.AreEqual(cookie, recoveredLog.RecoveredCookie); @@ -847,33 +846,27 @@ public void FasterLogManualCommitTest() log.Enqueue(entry); } - var commit1Addr = log.TailAddress; var cookie1 = new byte[100]; new Random().NextBytes(cookie1); - log.AddCommitCookie(cookie1); - log.CommitManuallyAtNum(1, true); + var commit1Addr = log.Commit(true, cookie1, 1); for (int i = 0; i < numEntries; i++) { log.Enqueue(entry); } - var commit2Addr = log.TailAddress; var cookie2 = new byte[100]; new Random().NextBytes(cookie2); - log.AddCommitCookie(cookie2); - log.CommitManuallyAtNum(2, true); + var commit2Addr = log.Commit(true, cookie2, 2); for (int i = 0; i < numEntries; i++) { log.Enqueue(entry); } - var commit6Addr = log.TailAddress; var cookie6 = new byte[100]; new Random().NextBytes(cookie6); - log.AddCommitCookie(cookie6); - log.CommitManuallyAtNum(6, true); + var commit6Addr = log.Commit(true, cookie6, 6); var recoveredLog = new FasterLog(logSettings, 1); Assert.AreEqual(cookie1, recoveredLog.RecoveredCookie); @@ -891,57 +884,5 @@ public void FasterLogManualCommitTest() Assert.AreEqual(commit6Addr, recoveredLog.TailAddress); recoveredLog.Dispose(); } - - [Test] - [Category("FasterLog")] - public void FasterLogMultiCommitCookieTest() - { - var cookies = new List(); - var cookieStatus = new List<(long, bool)>(); - for (var i = 0; i < 10; i++) - { - var cookie = new byte[100]; - new Random().NextBytes(cookie); - cookies.Add(cookie); - } - - device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; - log = new FasterLog(logSettings); - - byte[] entry = new byte[entryLength]; - for (int i = 0; i < entryLength; i++) - entry[i] = (byte)i; - - for (var j = 0; j < 10; j++) - { - for (int i = 0; i < numEntries; i++) - { - log.Enqueue(entry); - } - - var success = log.AddCommitCookie(cookies[j]); - cookieStatus.Add(ValueTuple.Create(log.TailAddress, success)); - } - - foreach (var commitNum in manager.ListCommits()) - { - var recoveredLog = new FasterLog(logSettings, commitNum); - var recoveredTail = recoveredLog.TailAddress; - - // Check that the closest successful cookie is recovered - byte[] expectedCookie = null; - for (var i = 0; i < 10; i++) - { - if (!cookieStatus[i].Item2) continue; - if (cookieStatus[i].Item1 > recoveredTail) break; - expectedCookie = cookies[i]; - } - - Assert.AreEqual(expectedCookie, recoveredLog.RecoveredCookie); - recoveredLog.Dispose(); - } - - } } } From c21aa078a25ff348771463cb33511e6478708793 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Tue, 26 Oct 2021 00:35:12 +0000 Subject: [PATCH 13/33] Prototype now works --- .../DeviceLogCommitCheckpointManager.cs | 25 ++--- cs/src/core/Index/FasterLog/FasterLog.cs | 100 +++++++++++------- .../Index/FasterLog/FasterLogRecoveryInfo.cs | 4 +- .../core/Index/FasterLog/ILogCommitManager.cs | 9 +- .../Index/FasterLog/LocalLogCommitManager.cs | 13 +-- cs/test/FasterLogTests.cs | 7 ++ 6 files changed, 97 insertions(+), 61 deletions(-) diff --git a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs index a82cdf2dd..f5abb1b16 100644 --- a/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs +++ b/cs/src/core/Index/CheckpointManagement/DeviceLogCommitCheckpointManager.cs @@ -29,11 +29,6 @@ public class DeviceLogCommitCheckpointManager : ILogCommitManager, ICheckpointMa private IDevice singleLogCommitDevice; private bool _disposed; - /// - /// Next commit number - /// - private long commitNum; - /// /// Track historical commits for automatic purging /// @@ -52,7 +47,6 @@ public DeviceLogCommitCheckpointManager(INamedDeviceFactory deviceFactory, IChec this.deviceFactory = deviceFactory; this.checkpointNamingScheme = checkpointNamingScheme; - this.commitNum = 0; this.semaphore = new SemaphoreSlim(0); this.overwriteLogCommits = overwriteLogCommits; @@ -99,10 +93,12 @@ public DeviceLogCommitCheckpointManager(INamedDeviceFactory deviceFactory, strin #region ILogCommitManager /// - public unsafe void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum = -1) + public bool PreciseCommitNumRecoverySupport() => !overwriteLogCommits; + + /// + public unsafe void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum) { - Debug.Assert(!overwriteLogCommits || overwriteLogCommits && proposedCommitNum == -1); - var device = NextCommitDevice(proposedCommitNum); + var device = NextCommitDevice(commitNum); if (device == null) return; @@ -139,6 +135,8 @@ public IEnumerable ListCommits() /// public void RemoveCommit(long commitNum) { + if (overwriteLogCommits) + throw new FasterException("removing commit by commit num is not supported when overwriting log commits"); deviceFactory.Delete(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); } @@ -170,7 +168,6 @@ public byte[] GetCommitMetadata(long commitNum) else { device = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); - this.commitNum = commitNum + 1; } if (device == null) return null; @@ -190,14 +187,14 @@ public byte[] GetCommitMetadata(long commitNum) return new Span(body).Slice(sizeof(int)).ToArray(); } - private IDevice NextCommitDevice(long proposedCommitNum) + private IDevice NextCommitDevice(long commitNum) { if (overwriteLogCommits) { if (_disposed) return null; if (singleLogCommitDevice == null) { - singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); + singleLogCommitDevice = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(0)); if (_disposed) { singleLogCommitDevice?.Dispose(); @@ -208,9 +205,7 @@ private IDevice NextCommitDevice(long proposedCommitNum) return singleLogCommitDevice; } - var actualNum = proposedCommitNum == -1 ? commitNum + 1 : proposedCommitNum; - var result = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(actualNum)); - commitNum = actualNum; + var result = deviceFactory.Get(checkpointNamingScheme.FasterLogCommitMetadata(commitNum)); return result; } #endregion diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index ebd64e976..7ef6901f8 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -129,7 +129,16 @@ internal readonly ConcurrentDictionary PersistedI public FasterLog(FasterLogSettings logSettings, long requestedCommitNum = -1) : this(logSettings, false) { - Restore(out var it, out RecoveredCookie, requestedCommitNum); + Dictionary it; + if (requestedCommitNum == -1) + RestoreLatest(out it, out RecoveredCookie); + else + { + if (!logCommitManager.PreciseCommitNumRecoverySupport()) + throw new FasterException("Recovering to a specific commit is not supported for given log setting"); + RestoreSpecificCommit(requestedCommitNum, out it, out RecoveredCookie); + } + RecoveredIterators = it; } @@ -1026,7 +1035,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) foreach (var recoveryInfo in coveredCommits) { - logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); + // logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); // Otherwise, set committed state as we commit metadata explicitly if (!fastCommitMode) UpdateCommittedState(recoveryInfo); @@ -1086,6 +1095,8 @@ private void UpdateTailCallback(long tailAddress) } } + + // TODO(Tianyu): Will we ever need to recover to a specific commit for read-only cases? /// /// Synchronously recover instance to FasterLog's latest commit, when being used as a readonly log iterator /// @@ -1094,7 +1105,7 @@ public void RecoverReadOnly() if (!readOnlyMode) throw new FasterException("This method can only be used with a read-only FasterLog instance used for iteration. Set FasterLogSettings.ReadOnlyMode to true during creation to indicate this."); - this.Restore(out _, out _); + this.RestoreLatest(out _, out _); SignalWaitingROIterators(); } @@ -1130,48 +1141,76 @@ private void SignalWaitingROIterators() _commitTcs?.TrySetResult(lci); } + private bool LoadCommitMetadata(long commitNum, out FasterLogRecoveryInfo info) + { + var commitInfo = logCommitManager.GetCommitMetadata(commitNum); + if (commitInfo is null) + { + info = default; + return false; + } + + info = new FasterLogRecoveryInfo(); + using (BinaryReader r = new(new MemoryStream(commitInfo))) + { + info.Initialize(r); + } + + return true; + } + private bool RestoreLatest(out Dictionary iterators, out byte[] cookie) { iterators = null; cookie = null; + FasterLogRecoveryInfo info = new(); - foreach (var commitNum in logCommitManager.ListCommits()) + foreach (var metadataCommit in logCommitManager.ListCommits()) { try { - if (RestoreFromCommitFile(commitNum, out iterators, out cookie)); + if (LoadCommitMetadata(metadataCommit, out info)) + break; } catch { } } - Debug.WriteLine("Unable to recover using any available commit"); - return false; - } - - /// - /// Restore log synchronously - /// - private bool RestoreCommitNum(long commitNum, out Dictionary iterators, out byte[] cookie) - { - throw new NotImplementedException(); - } - private bool RestoreFromCommitFile(long commitNum, out Dictionary iterators, out byte[] cookie) - { - iterators = null; - cookie = null; - if (!PrepareToRestoreFromCommit(commitNum, out var info, out var headAddress)) - { - Debug.WriteLine("Unable to recover using any specified commit num: " + commitNum); + // Shut up safe guards, I know what I am doing + CommittedUntilAddress = long.MaxValue; + allocator.HeadAddress = long.MaxValue; + using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); + scanIterator.ScanForwardForCommit(ref info); + if (info.UntilAddress == 0) + { + Debug.WriteLine("Unable to recover using any available commit"); + // Reset things to be something normal lol + allocator.Initialize(); + CommittedUntilAddress = Constants.kFirstValidAddress; return false; } + + if (!readOnlyMode) + { + var headAddress = info.UntilAddress - allocator.GetOffsetInPage(info.UntilAddress); + if (info.BeginAddress > headAddress) + headAddress = info.BeginAddress; - if (headAddress > 0) + if (headAddress == 0) + headAddress = Constants.kFirstValidAddress; allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress); + } iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; + return true; } + + + private bool RestoreSpecificCommit(long commitNum, out Dictionary iterators, out byte[] cookie) + { + throw new NotImplementedException(); + } /// /// Restore log asynchronously @@ -1199,18 +1238,7 @@ private async ValueTask> RestoreAsync(CancellationToken private bool PrepareToRestoreFromCommit(long commitNum, out FasterLogRecoveryInfo info, out long headAddress) { headAddress = 0; - var commitInfo = logCommitManager.GetCommitMetadata(commitNum); - if (commitInfo is null) - { - info = default; - return false; - } - - info = new FasterLogRecoveryInfo(); - using (BinaryReader r = new(new MemoryStream(commitInfo))) - { - info.Initialize(r); - } + if (!LoadCommitMetadata(commitNum, out info)) return false; if (!readOnlyMode) { diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index 3032756bd..0f88a09a5 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -64,6 +64,7 @@ public void Initialize(BinaryReader reader) checkSum = reader.ReadInt64(); BeginAddress = reader.ReadInt64(); UntilAddress = reader.ReadInt64(); + CommitNum = reader.ReadInt64(); } catch (Exception e) { @@ -121,6 +122,7 @@ public readonly byte[] ToByteArray() writer.Write(BeginAddress ^ UntilAddress); // checksum writer.Write(BeginAddress); writer.Write(UntilAddress); + writer.Write(CommitNum); if (Iterators?.Count > 0) { writer.Write(Iterators.Count); @@ -157,7 +159,7 @@ public int SerializedSize() iteratorSize += kvp.Key.Length + sizeof(long); } - return sizeof(int) + 3 * sizeof(long) + iteratorSize + Cookie?.Length ?? 0; + return sizeof(int) + 4 * sizeof(long) + iteratorSize + sizeof(int) + (Cookie?.Length ?? 0); } /// diff --git a/cs/src/core/Index/FasterLog/ILogCommitManager.cs b/cs/src/core/Index/FasterLog/ILogCommitManager.cs index 84fb1acb3..27a7b858c 100644 --- a/cs/src/core/Index/FasterLog/ILogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/ILogCommitManager.cs @@ -12,14 +12,17 @@ namespace FASTER.core /// public interface ILogCommitManager : IDisposable { + /// + /// Whether this log commit manager supports recovering to a specific commit num + bool PreciseCommitNumRecoverySupport(); + /// /// Perform (synchronous) commit with specified metadata /// /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - should be persisted - /// Proposed commit num, or -1 if none, in which case the implementation will pick one. - /// Caller must ensure commit num supplied is unique and monotonically increasing + /// commit num void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum); /// @@ -36,7 +39,7 @@ public interface ILogCommitManager : IDisposable public IEnumerable ListCommits(); /// - /// Remove the given commit, if present + /// Remove the given commit, if present. Should only be invoked if PreciseCommitNumRecoverySupport returns true /// /// commit num to remove public void RemoveCommit(long commitNum); diff --git a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs index a22269a57..9626ca0f3 100644 --- a/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs +++ b/cs/src/core/Index/FasterLog/LocalLogCommitManager.cs @@ -22,15 +22,18 @@ public LocalLogCommitManager(string commitFile) this.commitFile = commitFile; } + /// + public bool PreciseCommitNumRecoverySupport() => false; + + /// /// Perform (synchronous) commit with specified metadata /// /// Committed begin address (for information only, not necessary to persist) /// Address committed until (for information only, not necessary to persist) /// Commit metadata - /// Ignored param - - public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long proposedCommitNum) + /// Ignored param + public void Commit(long beginAddress, long untilAddress, byte[] commitMetadata, long commitNum) { // Two phase to ensure we write metadata in single Write operation @@ -81,9 +84,7 @@ public IEnumerable ListCommits() /// public void RemoveCommit(long commitNum) { - // we only use a single commit file in this implementation - if (commitNum != 0) return; - File.Delete(commitFile); + throw new FasterException("removing commit by commit num is not supported when overwriting log commits"); } /// diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 8f03eb41c..fa38cace0 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -822,6 +822,13 @@ public void FasterLogSimpleCommitCookieTest() log.Enqueue(entry); } + log.Commit(true, null); + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + log.Commit(true, cookie); var recoveredLog = new FasterLog(logSettings); From 23e329e0b44360eb248b431817e709259599ee6d Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Wed, 27 Oct 2021 14:39:19 +0000 Subject: [PATCH 14/33] fix to truncate log --- cs/src/core/Index/FasterLog/FasterLog.cs | 203 +++++++++++++----- .../core/Index/FasterLog/FasterLogIterator.cs | 14 +- cs/test/FasterLogTests.cs | 16 +- 3 files changed, 171 insertions(+), 62 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 7ef6901f8..b7e2d0129 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -45,7 +45,11 @@ private TaskCompletionSource refreshUncommittedTcs /// /// Beginning address of log /// - public long BeginAddress => allocator.BeginAddress; + public long BeginAddress => beginAddress; + + // Here's a soft begin address that is observed by all access at the FasterLog level but not actually on the + // allocator. This is to make sure that any potential physical deletes only happen after commit. + private long beginAddress; /// /// Tail address of log @@ -103,11 +107,6 @@ private TaskCompletionSource refreshUncommittedTcs internal readonly ConcurrentDictionary PersistedIterators = new ConcurrentDictionary(); - /// - /// Version number to track changes to commit metadata (begin address and persisted iterators) - /// - private long commitMetadataVersion; - /// /// Committed view of commitMetadataVersion /// @@ -449,7 +448,8 @@ public void WaitForCommit(long untilAddress = 0) var tailAddress = untilAddress; if (tailAddress == 0) tailAddress = allocator.GetTailAddress(); - while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) Thread.Yield(); + var observedCommitNum = commitNum; + while (CommittedUntilAddress < tailAddress || persistedCommitNum < observedCommitNum) Thread.Yield(); } /// @@ -467,13 +467,11 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok var tailAddress = untilAddress; if (tailAddress == 0) tailAddress = allocator.GetTailAddress(); - while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) + var observedCommitNum = commitNum; + while (CommittedUntilAddress < tailAddress || persistedCommitNum < observedCommitNum) { var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) - task = linkedCommitInfo.NextTask; - else - break; + task = linkedCommitInfo.NextTask; } } #endregion @@ -484,10 +482,41 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok /// Issue commit request for log (until tail) /// /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. - /// - public long Commit(bool spinWait = false, byte[] cookie = null, long commitNum = -1) + /// + /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from + /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog + /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed + /// + /// whether there is anything to commit. + + public bool Commit(bool spinWait = false, byte[] cookie = null) { - return CommitInternal(spinWait, cookie, commitNum); + return CommitInternal(out _, out _, spinWait); + } + + + /// + /// Issue commit request for log (until tail) with the given commitNum + /// + /// the tail committed by this call + /// + /// a unique, monotonically increasing identifier for the commit that can be used to recover to exactly + /// his commit + /// + /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. + /// + /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from + /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog + /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed + /// + /// /// + /// proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with + /// a non -1 value, commit is guaranteed to have the supplied identifier if commit call is successful + /// + /// whether commit is successful + public bool Commit(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) + { + return CommitInternal(out commitTail, out actualCommitNum, spinWait, cookie, proposedCommitNum); } /// @@ -496,21 +525,41 @@ public long Commit(bool spinWait = false, byte[] cookie = null, long commitNum = /// ongoing commit fails. /// /// - public async ValueTask CommitAsync(CancellationToken token = default, byte[] cookie = null, long commitNum = -1) + public async ValueTask CommitAsync(CancellationToken token = default, byte[] cookie = null) { token.ThrowIfCancellationRequested(); var task = CommitTask; - var tailAddress = CommitInternal(false, cookie, commitNum); + if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie)) + return; - while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) + while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) { var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) - task = linkedCommitInfo.NextTask; - else - break; + task = linkedCommitInfo.NextTask; } } + + /// + /// Async commit log (until tail), completes only when we + /// complete the commit. Throws exception if this or any + /// ongoing commit fails. + /// + /// + public async ValueTask CommitAsync(long proposedCommitNum, CancellationToken token = default, byte[] cookie = null) + { + token.ThrowIfCancellationRequested(); + var task = CommitTask; + if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie, proposedCommitNum)) + return false; + + while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) + { + var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); + task = linkedCommitInfo.NextTask; + } + + return true; + } /// /// Async commit log (until tail), completes only when we @@ -518,16 +567,18 @@ public async ValueTask CommitAsync(CancellationToken token = default, byte[] coo /// from prevCommitTask to current fails. /// /// - public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default, byte[] cookie = null, long commitNum = -1) + public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default, byte[] cookie = null) { token.ThrowIfCancellationRequested(); if (prevCommitTask == null) prevCommitTask = CommitTask; - var tailAddress = CommitInternal(false, cookie, commitNum); - while (CommittedUntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) + if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie)) + return prevCommitTask; + + while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) { var linkedCommitInfo = await prevCommitTask.WithCancellationAsync(token).ConfigureAwait(false); - if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < commitMetadataVersion) + if (linkedCommitInfo.CommitInfo.UntilAddress < tailAddress || persistedCommitNum < actualCommitNum) prevCommitTask = linkedCommitInfo.NextTask; else return linkedCommitInfo.NextTask; @@ -782,7 +833,7 @@ public async ValueTask EnqueueAndWaitForCommitAsync(IReadOnlySpanBatch rea /// Until address public void TruncateUntil(long untilAddress) { - allocator.ShiftBeginAddress(untilAddress); + Utility.MonotonicUpdate(ref beginAddress, untilAddress, out _); } /// @@ -793,7 +844,7 @@ public void TruncateUntil(long untilAddress) /// Until address public void TruncateUntilPageStart(long untilAddress) { - allocator.ShiftBeginAddress(untilAddress & ~allocator.PageSizeMask); + Utility.MonotonicUpdate(ref beginAddress, untilAddress & ~allocator.PageSizeMask, out _); } /// @@ -978,14 +1029,13 @@ private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info, out l private bool ShouldCommmitMetadata(ref FasterLogRecoveryInfo info) { - return allocator.BeginAddress > CommittedBeginAddress || IteratorsChanged(ref info) || info.Cookie != null; + return beginAddress > CommittedBeginAddress || IteratorsChanged(ref info) || info.Cookie != null; } private void CommitMetadataOnly(ref FasterLogRecoveryInfo info, bool spinWait) { - var beginAddress = allocator.BeginAddress; - var fromAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; - var untilAddress = CommittedUntilAddress > beginAddress ? CommittedUntilAddress : beginAddress; + var fromAddress = CommittedUntilAddress > info.BeginAddress ? CommittedUntilAddress : info.BeginAddress; + var untilAddress = CommittedUntilAddress > info.BeginAddress ? CommittedUntilAddress : info.BeginAddress; CommitCallback(new CommitInfo { @@ -1035,10 +1085,12 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) foreach (var recoveryInfo in coveredCommits) { - // logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); - // Otherwise, set committed state as we commit metadata explicitly + logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); + // Otherwise, set committed state as we commit metadata explicitly only after metadata commit if (!fastCommitMode) UpdateCommittedState(recoveryInfo); + // Issue any potential physical deletes due to shifts in begin address + allocator.ShiftBeginAddress(recoveryInfo.BeginAddress); } } @@ -1159,7 +1211,7 @@ private bool LoadCommitMetadata(long commitNum, out FasterLogRecoveryInfo info) return true; } - private bool RestoreLatest(out Dictionary iterators, out byte[] cookie) + private void RestoreLatest(out Dictionary iterators, out byte[] cookie) { iterators = null; cookie = null; @@ -1177,16 +1229,20 @@ private bool RestoreLatest(out Dictionary iterators, out byte[] co // Shut up safe guards, I know what I am doing CommittedUntilAddress = long.MaxValue; + beginAddress = info.BeginAddress; allocator.HeadAddress = long.MaxValue; using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); scanIterator.ScanForwardForCommit(ref info); + + // if until address is 0, that means info is still its default value and we haven't been able to recover + // from any any commit. Set the log to its start position and return if (info.UntilAddress == 0) { Debug.WriteLine("Unable to recover using any available commit"); // Reset things to be something normal lol allocator.Initialize(); CommittedUntilAddress = Constants.kFirstValidAddress; - return false; + return; } if (!readOnlyMode) @@ -1202,14 +1258,60 @@ private bool RestoreLatest(out Dictionary iterators, out byte[] co iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; - - return true; } - - private bool RestoreSpecificCommit(long commitNum, out Dictionary iterators, out byte[] cookie) + private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) { - throw new NotImplementedException(); + if (!logCommitManager.PreciseCommitNumRecoverySupport()) + throw new FasterException("Given FasterLog does not support recovery to a precise commit num"); + iterators = null; + cookie = null; + FasterLogRecoveryInfo info = new(); + + long scanStart = 0; + foreach (var metadataCommit in logCommitManager.ListCommits()) + { + if (metadataCommit > requestedCommitNum) continue; + try + { + if (LoadCommitMetadata(metadataCommit, out info)) + { + scanStart = metadataCommit; + break; + } + } + catch { } + } + + if (scanStart < requestedCommitNum) + { + // If no exact metadata is found, scan forward to see if we able to find a commit entry + // Shut up safe guards, I know what I am doing + CommittedUntilAddress = long.MaxValue; + beginAddress = info.BeginAddress; + allocator.HeadAddress = long.MaxValue; + using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); + if (!scanIterator.ScanForwardForCommit(ref info)) + // In the case where precisely requested commit num is not available, can just throw exception + throw new FasterException("requested commit num is not available"); + } + + // At this point, we should have found the exact commit num requested + Debug.Assert(info.CommitNum == requestedCommitNum); + if (!readOnlyMode) + { + var headAddress = info.UntilAddress - allocator.GetOffsetInPage(info.UntilAddress); + if (info.BeginAddress > headAddress) + headAddress = info.BeginAddress; + + if (headAddress == 0) + headAddress = Constants.kFirstValidAddress; + allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress); + } + + iterators = CompleteRestoreFromCommit(info); + cookie = info.Cookie; + } /// @@ -1450,8 +1552,10 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private long CommitInternal(bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) + private bool CommitInternal(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) { + commitTail = actualCommitNum = 0; + if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); @@ -1466,13 +1570,12 @@ private long CommitInternal(bool spinWait = false, byte[] cookie = null, long pr lock (outstandingCommitRecords) { if (proposedCommitNum == -1) - info.CommitNum = commitNum++; + info.CommitNum = actualCommitNum = ++commitNum; else if (proposedCommitNum > commitNum) - info.CommitNum = commitNum = proposedCommitNum; + info.CommitNum = actualCommitNum = commitNum = proposedCommitNum; else // Invalid commit num - return -1; - + return false; info.Cookie = cookie; do @@ -1481,7 +1584,7 @@ private long CommitInternal(bool spinWait = false, byte[] cookie = null, long pr if (CommittedUntilAddress == TailAddress && !ShouldCommmitMetadata(ref info)) { // Nothing to commit if no metadata update and no new entries, use -1 to denote that - return -1; + return false; } } while (!TryEnqueueCommitRecord(ref info, out commitRecordOffset)); @@ -1490,13 +1593,15 @@ private long CommitInternal(bool spinWait = false, byte[] cookie = null, long pr outstandingCommitRecords.Enqueue(ValueTuple.Create(commitRecordOffset, info)); } + commitTail = info.UntilAddress; + // Need to check, however, that a concurrent flush hasn't already advanced flushed address past this // commit. If so, need to manually trigger another commit callback in case the one triggered by the flush // already finished execution and missed our commit record if (commitRecordOffset < FlushedUntilAddress) { CommitMetadataOnly(ref info, spinWait); - return info.UntilAddress; + return true; } // Otherwise, move to set read-only tail and flush @@ -1517,7 +1622,7 @@ private long CommitInternal(bool spinWait = false, byte[] cookie = null, long pr // No need to trigger a manual commit even if no flush is triggered here -- if someone flushed the commit // record between our last check of FlushedUntilAddress and now, their flush callback would observe // this commitRecord in outstandingCommitRecords and commit it for us - return info.UntilAddress; + return true; } private long RefreshUncommittedInternal(bool spinWait = false) diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 9c06ecbe9..36cf6502f 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -378,7 +378,7 @@ private int Align(int length) return (length + 3) & ~3; } - internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info) + internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long commitNum = -1) { epoch.Resume(); var foundCommit = false; @@ -409,15 +409,23 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info) fixed (byte* bp = entry) Buffer.MemoryCopy((void*) (headerSize + physicalAddress), bp, entryLength, entryLength); info.Initialize(new BinaryReader(new MemoryStream(entry))); + + // If we have already found the commit number we are looking for, can stop early + if (info.CommitNum == commitNum) break; } } catch (FasterException) { // If we are here --- simply stop scanning because we ran into an incomplete entry } - epoch.Suspend(); - return foundCommit; + if (info.CommitNum == commitNum) + return true; + // User wants any commie + if (commitNum == -1) + return foundCommit; + // requested commit not found + return false; } /// diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index fa38cace0..38779b36f 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -822,13 +822,6 @@ public void FasterLogSimpleCommitCookieTest() log.Enqueue(entry); } - log.Commit(true, null); - - for (int i = 0; i < numEntries; i++) - { - log.Enqueue(entry); - } - log.Commit(true, cookie); var recoveredLog = new FasterLog(logSettings); @@ -855,7 +848,8 @@ public void FasterLogManualCommitTest() var cookie1 = new byte[100]; new Random().NextBytes(cookie1); - var commit1Addr = log.Commit(true, cookie1, 1); + var commitSuccessful = log.Commit(out var commit1Addr, out _, true, cookie1, 1); + Assert.IsTrue(commitSuccessful); for (int i = 0; i < numEntries; i++) { @@ -864,7 +858,8 @@ public void FasterLogManualCommitTest() var cookie2 = new byte[100]; new Random().NextBytes(cookie2); - var commit2Addr = log.Commit(true, cookie2, 2); + commitSuccessful = log.Commit(out var commit2Addr, out _, true, cookie2, 2); + Assert.IsTrue(commitSuccessful); for (int i = 0; i < numEntries; i++) { @@ -873,7 +868,8 @@ public void FasterLogManualCommitTest() var cookie6 = new byte[100]; new Random().NextBytes(cookie6); - var commit6Addr = log.Commit(true, cookie6, 6); + commitSuccessful = log.Commit(out var commit6Addr, out _, true, cookie6, 6); + Assert.IsTrue(commitSuccessful); var recoveredLog = new FasterLog(logSettings, 1); Assert.AreEqual(cookie1, recoveredLog.RecoveredCookie); From 94b12c7432d9fa04d69ab9a7311bbbc5f91f0e3c Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Wed, 27 Oct 2021 15:31:25 +0000 Subject: [PATCH 15/33] Add condition to only scan log in fastCommitMode --- cs/src/core/Index/FasterLog/FasterLog.cs | 33 ++++++++++++++++-------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index b7e2d0129..6b9c27087 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1227,13 +1227,17 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co catch { } } - // Shut up safe guards, I know what I am doing - CommittedUntilAddress = long.MaxValue; - beginAddress = info.BeginAddress; - allocator.HeadAddress = long.MaxValue; - using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); - scanIterator.ScanForwardForCommit(ref info); - + // Only in fast commit mode will we potentially need to recover from an entry in the log + if (fastCommitMode) + { + // Shut up safe guards, I know what I am doing + CommittedUntilAddress = long.MaxValue; + beginAddress = info.BeginAddress; + allocator.HeadAddress = long.MaxValue; + using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); + scanIterator.ScanForwardForCommit(ref info); + } + // if until address is 0, that means info is still its default value and we haven't been able to recover // from any any commit. Set the log to its start position and return if (info.UntilAddress == 0) @@ -1258,6 +1262,7 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; + commitNum = info.CommitNum; } private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) @@ -1268,6 +1273,7 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary From eca8ef3c28ae910775bcf0a48fc7a4d15946e865 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 28 Oct 2021 01:00:01 +0000 Subject: [PATCH 16/33] Rough draft to allow for 1. elided commit record when fastCommit turned off and 2. "loose" commits --- cs/src/core/Index/FasterLog/FasterLog.cs | 131 ++++++++++++------ .../core/Index/FasterLog/FasterLogIterator.cs | 4 +- cs/test/FasterLogTests.cs | 4 +- 3 files changed, 92 insertions(+), 47 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 6b9c27087..ac286f719 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -21,6 +21,8 @@ namespace FASTER.core /// public class FasterLog : IDisposable { + public const long AUTO_COMMIT_NUM = -1; + public const long NO_COMMIT_NUM = -2; private readonly BlittableAllocator allocator; private readonly LightEpoch epoch; private readonly ILogCommitManager logCommitManager; @@ -39,7 +41,7 @@ private TaskCompletionSource refreshUncommittedTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); // Offsets for all currently unprocessed commit records - private Queue<(long, FasterLogRecoveryInfo)> outstandingCommitRecords; + private Queue<(long, FasterLogRecoveryInfo)> ongoingCommitRequests; private long commitNum; /// @@ -56,6 +58,9 @@ private TaskCompletionSource refreshUncommittedTcs /// public long TailAddress => allocator.GetTailAddress(); + // Used to track the last user entry, to stop commits from committing only other commit records + private long userEntryTailAddress, lastProcessedUserEntryTailAddress; + /// /// Log flushed until address /// @@ -189,7 +194,7 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) fastCommitMode = logSettings.FastCommitMode; - outstandingCommitRecords = new Queue<(long, FasterLogRecoveryInfo)>(); + ongoingCommitRequests = new Queue<(long, FasterLogRecoveryInfo)>(); } /// @@ -275,6 +280,8 @@ public unsafe bool TryEnqueue(byte[] entry, out long logicalAddress) return false; } + Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); fixed (byte* bp = entry) Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); @@ -306,6 +313,8 @@ public unsafe bool TryEnqueue(ReadOnlySpan entry, out long logicalAddress) return false; } + Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); fixed (byte* bp = &entry.GetPinnableReference()) Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); @@ -992,7 +1001,7 @@ private int Align(int length) private void CommitCallback(CommitInfo commitInfo) { // Using count is safe as a fast filtering mechanism to reduce number of invocations despite concurrency - if (outstandingCommitRecords.Count == 0) return; + if (ongoingCommitRequests.Count == 0) return; commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } @@ -1046,7 +1055,7 @@ private void CommitMetadataOnly(ref FasterLogRecoveryInfo info, bool spinWait) if (spinWait) { - while (info.CommitNum < persistedCommitNum) + while (Math.Abs(info.CommitNum) < persistedCommitNum) Thread.Yield(); } } @@ -1057,7 +1066,19 @@ private void UpdateCommittedState(FasterLogRecoveryInfo recoveryInfo) CommittedBeginAddress = recoveryInfo.BeginAddress; CommittedUntilAddress = recoveryInfo.UntilAddress; recoveryInfo.CommitIterators(PersistedIterators); - Utility.MonotonicUpdate(ref persistedCommitNum, recoveryInfo.CommitNum, out _); + Utility.MonotonicUpdate(ref persistedCommitNum, Math.Abs(recoveryInfo.CommitNum), out _); + } + + private void WriteCommitMetadata(FasterLogRecoveryInfo recoveryInfo) + { + // TODO(Tianyu): If fast commit, write this in separate thread? + logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, + recoveryInfo.ToByteArray(), Math.Abs(recoveryInfo.CommitNum)); + // If not fast committing, set committed state as we commit metadata explicitly only after metadata commit + if (!fastCommitMode) + UpdateCommittedState(recoveryInfo); + // Issue any potential physical deletes due to shifts in begin address + allocator.ShiftBeginAddress(recoveryInfo.BeginAddress); } private void SerialCommitCallbackWorker(CommitInfo commitInfo) @@ -1066,32 +1087,34 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) { var coveredCommits = new List(); // Check for the commit records included in this flush - lock (outstandingCommitRecords) + lock (ongoingCommitRequests) { - while (outstandingCommitRecords.Count != 0) + while (ongoingCommitRequests.Count != 0) { - var (addr, recoveryInfo) = outstandingCommitRecords.Peek(); + var (addr, recoveryInfo) = ongoingCommitRequests.Peek(); if (addr >= commitInfo.UntilAddress) break; coveredCommits.Add(recoveryInfo); - outstandingCommitRecords.Dequeue(); + ongoingCommitRequests.Dequeue(); } } + // Nothing was committed --- this was probably au auto-flush. Return now without touching any + // commit task tracking. + if (coveredCommits.Count == 0) return; + + var latestCommit = coveredCommits[coveredCommits.Count - 1]; if (fastCommitMode) - { // In fast commit mode, can safely set committed state to the latest flushed - UpdateCommittedState(coveredCommits[coveredCommits.Count - 1]); - } + UpdateCommittedState(latestCommit); foreach (var recoveryInfo in coveredCommits) { - logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); - // Otherwise, set committed state as we commit metadata explicitly only after metadata commit - if (!fastCommitMode) - UpdateCommittedState(recoveryInfo); - // Issue any potential physical deletes due to shifts in begin address - allocator.ShiftBeginAddress(recoveryInfo.BeginAddress); + // Only write out commit metadata if user cares about this as a distinct recoverable point + if (recoveryInfo.CommitNum >= 0) WriteCommitMetadata(recoveryInfo); } + + // We skipped any NO_COMMIT_NUM commits earlier, so write it out if not covered by another commit + if (latestCommit.CommitNum < 0) WriteCommitMetadata(latestCommit); } var _commitTcs = commitTcs; @@ -1262,7 +1285,7 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; - commitNum = info.CommitNum; + commitNum = Math.Abs(info.CommitNum); } private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) @@ -1308,7 +1331,7 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary @@ -1407,6 +1430,8 @@ private unsafe bool TryAppend(IReadOnlySpanBatch readOnlySpanBatch, out long log epoch.Resume(); logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); + Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); + if (logicalAddress == 0) { epoch.Suspend(); @@ -1563,7 +1588,7 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private bool CommitInternal(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) + private bool CommitInternal(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = NO_COMMIT_NUM) { commitTail = actualCommitNum = 0; @@ -1572,44 +1597,64 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool // set the content of this commit to the current tail and base all commit metadata on this address, even // though perhaps more entries will be flushed as part of this commit - long commitRecordOffset; + long commitAddress; var info = new FasterLogRecoveryInfo(); - - - // This critical section serializes commit record creation and ensures that the long address are sorted in - // outstandingCommitRecords. Ok because we do not expect heavy contention on the commit code path - lock (outstandingCommitRecords) + + // This critical section serializes commit record creation / commit content generation and ensures that the + // long address are sorted in outstandingCommitRecords. Ok because we do not expect heavy contention on the + // commit code path + lock (ongoingCommitRequests) { - if (proposedCommitNum == -1) + // Compute regular information about the commit + info.Cookie = cookie; + info.SnapshotIterators(PersistedIterators); + + if (userEntryTailAddress == lastProcessedUserEntryTailAddress && !ShouldCommmitMetadata(ref info)) + // Nothing to commit if no metadata update and no new entries, use -1 to denote that + return false; + + + if (proposedCommitNum == AUTO_COMMIT_NUM) info.CommitNum = actualCommitNum = ++commitNum; + else if (proposedCommitNum == NO_COMMIT_NUM) + // We still need to generate a unique commit num so we can track metadata version, but we use the + // sign but to indicate that this commit can be fast-forwarded and not necessarily written out + info.CommitNum = -(++commitNum); else if (proposedCommitNum > commitNum) info.CommitNum = actualCommitNum = commitNum = proposedCommitNum; else // Invalid commit num return false; - info.Cookie = cookie; - do + // Mark all previous userEntryTailAddress as processed. Even though there are races and we might end + // up committing more than we marked, this restricts commits to only actually proceed past this check a + // often as there are new user entries. + lastProcessedUserEntryTailAddress = userEntryTailAddress; + + if (fastCommitMode) { - info.SnapshotIterators(PersistedIterators); - if (CommittedUntilAddress == TailAddress && !ShouldCommmitMetadata(ref info)) - { - // Nothing to commit if no metadata update and no new entries, use -1 to denote that - return false; - } - } while (!TryEnqueueCommitRecord(ref info, out commitRecordOffset)); - + // Ok to retry in critical section, any concurrently invoked commit would block, but cannot progress + // anyways if no record can be enqueued + while (!TryEnqueueCommitRecord(ref info, out commitTail)) {} + } + else + { + // If not using fastCommitMode, do not need to allocate a commit record. Instead, set committed + // address to current tail + info.BeginAddress = BeginAddress; + info.UntilAddress = commitTail = TailAddress; + } + // Enqueue the commit record's content and offset into the queue so it can be picked up by the next flush // At this point, we expect the commit record to be flushed out as a distinct recovery point - outstandingCommitRecords.Enqueue(ValueTuple.Create(commitRecordOffset, info)); + ongoingCommitRequests.Enqueue(ValueTuple.Create(commitTail, info)); } - commitTail = info.UntilAddress; // Need to check, however, that a concurrent flush hasn't already advanced flushed address past this // commit. If so, need to manually trigger another commit callback in case the one triggered by the flush // already finished execution and missed our commit record - if (commitRecordOffset < FlushedUntilAddress) + if (commitTail < FlushedUntilAddress) { CommitMetadataOnly(ref info, spinWait); return true; @@ -1622,7 +1667,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { if (spinWait) { - while (CommittedUntilAddress < commitRecordOffset) + while (CommittedUntilAddress < commitTail) { epoch.ProtectAndDrain(); Thread.Yield(); diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 36cf6502f..9c9223069 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -411,7 +411,7 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c info.Initialize(new BinaryReader(new MemoryStream(entry))); // If we have already found the commit number we are looking for, can stop early - if (info.CommitNum == commitNum) break; + if (Math.Abs(info.CommitNum) == commitNum) break; } } catch (FasterException) @@ -419,7 +419,7 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c // If we are here --- simply stop scanning because we ran into an incomplete entry } epoch.Suspend(); - if (info.CommitNum == commitNum) + if (Math.Abs(info.CommitNum) == commitNum) return true; // User wants any commie if (commitNum == -1) diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 38779b36f..fbf671ee9 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -804,13 +804,13 @@ internal class FasterLogCustomCommitTests : FasterLogTestBase [Test] [Category("FasterLog")] [Category("Smoke")] - public void FasterLogSimpleCommitCookieTest() + public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) { var cookie = new byte[100]; new Random().NextBytes(cookie); device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = fastCommit}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; From ab3be95351e4cd6eaad5758251b0aa7fba276576 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 28 Oct 2021 20:40:46 +0000 Subject: [PATCH 17/33] checkpoint changes --- cs/src/core/Index/FasterLog/FasterLog.cs | 232 +++++++++++------- .../core/Index/FasterLog/FasterLogIterator.cs | 4 +- .../Index/FasterLog/FasterLogRecoveryInfo.cs | 2 + cs/test/FasterLogTests.cs | 8 +- 4 files changed, 146 insertions(+), 100 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index ac286f719..3da535dd3 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -21,8 +21,6 @@ namespace FASTER.core /// public class FasterLog : IDisposable { - public const long AUTO_COMMIT_NUM = -1; - public const long NO_COMMIT_NUM = -2; private readonly BlittableAllocator allocator; private readonly LightEpoch epoch; private readonly ILogCommitManager logCommitManager; @@ -58,8 +56,9 @@ private TaskCompletionSource refreshUncommittedTcs /// public long TailAddress => allocator.GetTailAddress(); - // Used to track the last user entry, to stop commits from committing only other commit records - private long userEntryTailAddress, lastProcessedUserEntryTailAddress; + // Used to track the last commit record and commits that have been issued, to stop commits from committing + // without any user records + private long commitCoveredAddress; /// /// Log flushed until address @@ -154,7 +153,9 @@ public FasterLog(FasterLogSettings logSettings, long requestedCommitNum = -1) public static async ValueTask CreateAsync(FasterLogSettings logSettings, CancellationToken cancellationToken = default) { var fasterLog = new FasterLog(logSettings, false); - fasterLog.RecoveredIterators = await fasterLog.RestoreAsync(cancellationToken).ConfigureAwait(false); + var (it, cookie) = await fasterLog.RestoreLatestAsync(cancellationToken).ConfigureAwait(false); + fasterLog.RecoveredIterators = it; + fasterLog.RecoveredCookie = cookie; return fasterLog; } @@ -184,6 +185,8 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) logSettings.GetLogSettings(), null, null, epoch, CommitCallback); allocator.Initialize(); + // TODO(Tianyu): Move to track commit record tail rather than user tail + beginAddress = allocator.BeginAddress; // FasterLog is used as a read-only iterator if (logSettings.ReadOnlyMode) @@ -279,9 +282,7 @@ public unsafe bool TryEnqueue(byte[] entry, out long logicalAddress) epoch.Suspend(); return false; } - - Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); - + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); fixed (byte* bp = entry) Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); @@ -312,9 +313,7 @@ public unsafe bool TryEnqueue(ReadOnlySpan entry, out long logicalAddress) epoch.Suspend(); return false; } - - Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); - + var physicalAddress = allocator.GetPhysicalAddress(logicalAddress); fixed (byte* bp = &entry.GetPinnableReference()) Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), length, length); @@ -491,19 +490,13 @@ public async ValueTask WaitForCommitAsync(long untilAddress = 0, CancellationTok /// Issue commit request for log (until tail) /// /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. - /// - /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from - /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog - /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed - /// /// whether there is anything to commit. - public bool Commit(bool spinWait = false, byte[] cookie = null) + public void Commit(bool spinWait = false) { - return CommitInternal(out _, out _, spinWait); + CommitInternal(out _, out _, spinWait, true, null, -1); } - - + /// /// Issue commit request for log (until tail) with the given commitNum /// @@ -523,9 +516,9 @@ public bool Commit(bool spinWait = false, byte[] cookie = null) /// a non -1 value, commit is guaranteed to have the supplied identifier if commit call is successful /// /// whether commit is successful - public bool Commit(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) + public bool CommitStrongly(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) { - return CommitInternal(out commitTail, out actualCommitNum, spinWait, cookie, proposedCommitNum); + return CommitInternal(out commitTail, out actualCommitNum, spinWait, false, cookie, proposedCommitNum); } /// @@ -534,11 +527,11 @@ public bool Commit(out long commitTail, out long actualCommitNum, bool spinWait /// ongoing commit fails. /// /// - public async ValueTask CommitAsync(CancellationToken token = default, byte[] cookie = null) + public async ValueTask CommitAsync(CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; - if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie)) + if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, true, null, -1)) return; while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) @@ -547,28 +540,6 @@ public async ValueTask CommitAsync(CancellationToken token = default, byte[] coo task = linkedCommitInfo.NextTask; } } - - /// - /// Async commit log (until tail), completes only when we - /// complete the commit. Throws exception if this or any - /// ongoing commit fails. - /// - /// - public async ValueTask CommitAsync(long proposedCommitNum, CancellationToken token = default, byte[] cookie = null) - { - token.ThrowIfCancellationRequested(); - var task = CommitTask; - if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie, proposedCommitNum)) - return false; - - while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) - { - var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); - task = linkedCommitInfo.NextTask; - } - - return true; - } /// /// Async commit log (until tail), completes only when we @@ -576,12 +547,12 @@ public async ValueTask CommitAsync(long proposedCommitNum, CancellationTok /// from prevCommitTask to current fails. /// /// - public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default, byte[] cookie = null) + public async ValueTask> CommitAsync(Task prevCommitTask, CancellationToken token = default) { token.ThrowIfCancellationRequested(); if (prevCommitTask == null) prevCommitTask = CommitTask; - if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, cookie)) + if (!CommitInternal(out var tailAddress, out var actualCommitNum, false, true, null, -1)) return prevCommitTask; while (CommittedUntilAddress < tailAddress || persistedCommitNum < actualCommitNum) @@ -595,6 +566,41 @@ public async ValueTask> CommitAsync(Task + /// Issue commit request for log (until tail) with the given commitNum + /// + /// the tail committed by this call + /// + /// a unique, monotonically increasing identifier for the commit that can be used to recover to exactly + /// his commit + /// + /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. + /// + /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from + /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog + /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed + /// + /// /// + /// proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with + /// a non -1 value, commit is guaranteed to have the supplied identifier if commit call is successful + /// + /// whether commit is successful + public async ValueTask<(bool, long, long)> CommitStronglyAsync(bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1, CancellationToken token = default) + { + token.ThrowIfCancellationRequested(); + var task = CommitTask; + if (!CommitInternal(out var commitTail, out var actualCommitNum, false, false, cookie, proposedCommitNum)) + return ValueTuple.Create(false, commitTail, actualCommitNum); + + while (CommittedUntilAddress < commitTail || persistedCommitNum < actualCommitNum) + { + var linkedCommitInfo = await task.WithCancellationAsync(token).ConfigureAwait(false); + task = linkedCommitInfo.NextTask; + } + + return ValueTuple.Create(true, commitTail, actualCommitNum); + } /// /// Trigger a refresh of information about uncommitted part of log (tail address) to ensure visibility @@ -1005,10 +1011,8 @@ private void CommitCallback(CommitInfo commitInfo) commitQueue.EnqueueAndTryWork(commitInfo, asTask: true); } - private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info, out long logicalAddress) + private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info) { - logicalAddress = 0; - var entryBodySize = info.SerializedSize(); int allocatedLength = headerSize + Align(entryBodySize); @@ -1016,7 +1020,7 @@ private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info, out l epoch.Resume(); - logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); + var logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); if (logicalAddress == 0) { epoch.Suspend(); @@ -1033,6 +1037,7 @@ private unsafe bool TryEnqueueCommitRecord(ref FasterLogRecoveryInfo info, out l Buffer.MemoryCopy(bp, (void*)(headerSize + physicalAddress), entryBody.Length, entryBody.Length); SetCommitRecordHeader(entryBody.Length, (byte*)physicalAddress); epoch.Suspend(); + // Return the commit tail return true; } @@ -1055,7 +1060,7 @@ private void CommitMetadataOnly(ref FasterLogRecoveryInfo info, bool spinWait) if (spinWait) { - while (Math.Abs(info.CommitNum) < persistedCommitNum) + while (info.CommitNum < persistedCommitNum) Thread.Yield(); } } @@ -1066,14 +1071,14 @@ private void UpdateCommittedState(FasterLogRecoveryInfo recoveryInfo) CommittedBeginAddress = recoveryInfo.BeginAddress; CommittedUntilAddress = recoveryInfo.UntilAddress; recoveryInfo.CommitIterators(PersistedIterators); - Utility.MonotonicUpdate(ref persistedCommitNum, Math.Abs(recoveryInfo.CommitNum), out _); + Utility.MonotonicUpdate(ref persistedCommitNum, recoveryInfo.CommitNum, out _); } private void WriteCommitMetadata(FasterLogRecoveryInfo recoveryInfo) { // TODO(Tianyu): If fast commit, write this in separate thread? logCommitManager.Commit(recoveryInfo.BeginAddress, recoveryInfo.UntilAddress, - recoveryInfo.ToByteArray(), Math.Abs(recoveryInfo.CommitNum)); + recoveryInfo.ToByteArray(), recoveryInfo.CommitNum); // If not fast committing, set committed state as we commit metadata explicitly only after metadata commit if (!fastCommitMode) UpdateCommittedState(recoveryInfo); @@ -1092,7 +1097,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) while (ongoingCommitRequests.Count != 0) { var (addr, recoveryInfo) = ongoingCommitRequests.Peek(); - if (addr >= commitInfo.UntilAddress) break; + if (addr > commitInfo.UntilAddress) break; coveredCommits.Add(recoveryInfo); ongoingCommitRequests.Dequeue(); } @@ -1110,11 +1115,11 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) foreach (var recoveryInfo in coveredCommits) { // Only write out commit metadata if user cares about this as a distinct recoverable point - if (recoveryInfo.CommitNum >= 0) WriteCommitMetadata(recoveryInfo); + if (!recoveryInfo.FastForwardAllowed) WriteCommitMetadata(recoveryInfo); } - // We skipped any NO_COMMIT_NUM commits earlier, so write it out if not covered by another commit - if (latestCommit.CommitNum < 0) WriteCommitMetadata(latestCommit); + // We fast-forwarded commits earlier, so write it out if not covered by another commit + if (latestCommit.FastForwardAllowed) WriteCommitMetadata(latestCommit); } var _commitTcs = commitTcs; @@ -1136,7 +1141,7 @@ private bool IteratorsChanged(ref FasterLogRecoveryInfo info) var _lastPersistedIterators = LastPersistedIterators; if (_lastPersistedIterators == null) { - return info.Iterators.Count != 0; + return info.Iterators != null && info.Iterators.Count != 0; } if (_lastPersistedIterators.Count != info.Iterators.Count) return true; @@ -1192,7 +1197,7 @@ public async ValueTask RecoverReadOnlyAsync(CancellationToken cancellationToken if (!readOnlyMode) throw new FasterException("This method can only be used with a read-only FasterLog instance used for iteration. Set FasterLogSettings.ReadOnlyMode to true during creation to indicate this."); - await this.RestoreAsync(cancellationToken).ConfigureAwait(false); + await this.RestoreLatestAsync(cancellationToken).ConfigureAwait(false); SignalWaitingROIterators(); } @@ -1269,6 +1274,7 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co // Reset things to be something normal lol allocator.Initialize(); CommittedUntilAddress = Constants.kFirstValidAddress; + beginAddress = allocator.BeginAddress; return; } @@ -1285,7 +1291,8 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co iterators = CompleteRestoreFromCommit(info); cookie = info.Cookie; - commitNum = Math.Abs(info.CommitNum); + commitNum = info.CommitNum; + beginAddress = allocator.BeginAddress; } private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) @@ -1331,7 +1338,7 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary /// Restore log asynchronously /// - private async ValueTask> RestoreAsync(CancellationToken cancellationToken) + private async ValueTask<(Dictionary, byte[])> RestoreLatestAsync(CancellationToken cancellationToken) { - foreach (var commitNum in logCommitManager.ListCommits()) + FasterLogRecoveryInfo info = new(); + + foreach (var metadataCommit in logCommitManager.ListCommits()) { try { - if (!PrepareToRestoreFromCommit(commitNum, out FasterLogRecoveryInfo info, out long headAddress)) - return default; - - if (headAddress > 0) - await allocator.RestoreHybridLogAsync(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress, cancellationToken: cancellationToken).ConfigureAwait(false); - - return CompleteRestoreFromCommit(info); + if (LoadCommitMetadata(metadataCommit, out info)) + break; } catch { } } - Debug.WriteLine("Unable to recover using any available commit"); - return null; + + // Only in fast commit mode will we potentially need to recover from an entry in the log + if (fastCommitMode) + { + // Shut up safe guards, I know what I am doing + CommittedUntilAddress = long.MaxValue; + beginAddress = info.BeginAddress; + allocator.HeadAddress = long.MaxValue; + using var scanIterator = Scan(info.UntilAddress, long.MaxValue, recover: false); + scanIterator.ScanForwardForCommit(ref info); + } + + // if until address is 0, that means info is still its default value and we haven't been able to recover + // from any any commit. Set the log to its start position and return + if (info.UntilAddress == 0) + { + Debug.WriteLine("Unable to recover using any available commit"); + // Reset things to be something normal lol + allocator.Initialize(); + CommittedUntilAddress = Constants.kFirstValidAddress; + beginAddress = allocator.BeginAddress; + return ValueTuple.Create, byte[]>(new Dictionary(), null); + } + + if (!readOnlyMode) + { + var headAddress = info.UntilAddress - allocator.GetOffsetInPage(info.UntilAddress); + if (info.BeginAddress > headAddress) + headAddress = info.BeginAddress; + + if (headAddress == 0) + headAddress = Constants.kFirstValidAddress; + await allocator.RestoreHybridLogAsync(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress, cancellationToken : cancellationToken).ConfigureAwait(false); + } + + var iterators = CompleteRestoreFromCommit(info); + var cookie = info.Cookie; + commitNum = info.CommitNum; + beginAddress = allocator.BeginAddress; + return ValueTuple.Create(iterators, cookie); } private bool PrepareToRestoreFromCommit(long commitNum, out FasterLogRecoveryInfo info, out long headAddress) @@ -1430,7 +1473,6 @@ private unsafe bool TryAppend(IReadOnlySpanBatch readOnlySpanBatch, out long log epoch.Resume(); logicalAddress = allocator.TryAllocateRetryNow(allocatedLength); - Utility.MonotonicUpdate(ref userEntryTailAddress, logicalAddress + allocatedLength, out _); if (logicalAddress == 0) { @@ -1588,17 +1630,20 @@ private int GetRecordLengthAndFree(SectorAlignedMemory record) return length; } - private bool CommitInternal(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = NO_COMMIT_NUM) + private bool CommitInternal(out long commitTail, out long actualCommitNum, bool spinWait, bool allowFastForward, byte[] cookie, long proposedCommitNum) { commitTail = actualCommitNum = 0; if (readOnlyMode) throw new FasterException("Cannot commit in read-only mode"); - // set the content of this commit to the current tail and base all commit metadata on this address, even - // though perhaps more entries will be flushed as part of this commit - long commitAddress; + if (allowFastForward && (cookie != null || proposedCommitNum != -1)) + throw new FasterException( + "Fast forwarding a commit is only allowed when no cookie and not commit num is specified"); + + var info = new FasterLogRecoveryInfo(); + info.FastForwardAllowed = allowFastForward; // This critical section serializes commit record creation / commit content generation and ensures that the // long address are sorted in outstandingCommitRecords. Ok because we do not expect heavy contention on the @@ -1609,41 +1654,36 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool info.Cookie = cookie; info.SnapshotIterators(PersistedIterators); - if (userEntryTailAddress == lastProcessedUserEntryTailAddress && !ShouldCommmitMetadata(ref info)) - // Nothing to commit if no metadata update and no new entries, use -1 to denote that + if (commitCoveredAddress == TailAddress && !ShouldCommmitMetadata(ref info)) + // Nothing to commit if no metadata update and no new entries return false; - if (proposedCommitNum == AUTO_COMMIT_NUM) + if (proposedCommitNum == -1) info.CommitNum = actualCommitNum = ++commitNum; - else if (proposedCommitNum == NO_COMMIT_NUM) - // We still need to generate a unique commit num so we can track metadata version, but we use the - // sign but to indicate that this commit can be fast-forwarded and not necessarily written out - info.CommitNum = -(++commitNum); else if (proposedCommitNum > commitNum) info.CommitNum = actualCommitNum = commitNum = proposedCommitNum; else // Invalid commit num return false; - // Mark all previous userEntryTailAddress as processed. Even though there are races and we might end - // up committing more than we marked, this restricts commits to only actually proceed past this check a - // often as there are new user entries. - lastProcessedUserEntryTailAddress = userEntryTailAddress; if (fastCommitMode) { // Ok to retry in critical section, any concurrently invoked commit would block, but cannot progress // anyways if no record can be enqueued - while (!TryEnqueueCommitRecord(ref info, out commitTail)) {} + while (!TryEnqueueCommitRecord(ref info)) {} + commitTail = info.UntilAddress; } else { - // If not using fastCommitMode, do not need to allocate a commit record. Instead, set committed - // address to current tail + // If not using fastCommitMode, do not need to allocate a commit record. Instead, set the content + // of this commit to the current tail and base all commit metadata on this address, even though + // perhaps more entries will be flushed as part of this commit info.BeginAddress = BeginAddress; info.UntilAddress = commitTail = TailAddress; } + Utility.MonotonicUpdate(ref commitCoveredAddress, commitTail, out _); // Enqueue the commit record's content and offset into the queue so it can be picked up by the next flush // At this point, we expect the commit record to be flushed out as a distinct recovery point @@ -1674,6 +1714,10 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool } } } + else + { + CommitMetadataOnly(ref info, spinWait); + } epoch.Suspend(); // No need to trigger a manual commit even if no flush is triggered here -- if someone flushed the commit // record between our last check of FlushedUntilAddress and now, their flush callback would observe diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 9c9223069..36cf6502f 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -411,7 +411,7 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c info.Initialize(new BinaryReader(new MemoryStream(entry))); // If we have already found the commit number we are looking for, can stop early - if (Math.Abs(info.CommitNum) == commitNum) break; + if (info.CommitNum == commitNum) break; } } catch (FasterException) @@ -419,7 +419,7 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c // If we are here --- simply stop scanning because we ran into an incomplete entry } epoch.Suspend(); - if (Math.Abs(info.CommitNum) == commitNum) + if (info.CommitNum == commitNum) return true; // User wants any commie if (commitNum == -1) diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index 0f88a09a5..c61933183 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -37,6 +37,8 @@ internal struct FasterLogRecoveryInfo public byte[] Cookie; public long CommitNum; + + public bool FastForwardAllowed; /// /// Initialize diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index fbf671ee9..4c1b2ff4f 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -822,7 +822,7 @@ public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) log.Enqueue(entry); } - log.Commit(true, cookie); + log.CommitStrongly(out _, out _, true, cookie); var recoveredLog = new FasterLog(logSettings); Assert.AreEqual(cookie, recoveredLog.RecoveredCookie); @@ -848,7 +848,7 @@ public void FasterLogManualCommitTest() var cookie1 = new byte[100]; new Random().NextBytes(cookie1); - var commitSuccessful = log.Commit(out var commit1Addr, out _, true, cookie1, 1); + var commitSuccessful = log.CommitStrongly(out var commit1Addr, out _, true, cookie1, 1); Assert.IsTrue(commitSuccessful); for (int i = 0; i < numEntries; i++) @@ -858,7 +858,7 @@ public void FasterLogManualCommitTest() var cookie2 = new byte[100]; new Random().NextBytes(cookie2); - commitSuccessful = log.Commit(out var commit2Addr, out _, true, cookie2, 2); + commitSuccessful = log.CommitStrongly(out var commit2Addr, out _, true, cookie2, 2); Assert.IsTrue(commitSuccessful); for (int i = 0; i < numEntries; i++) @@ -868,7 +868,7 @@ public void FasterLogManualCommitTest() var cookie6 = new byte[100]; new Random().NextBytes(cookie6); - commitSuccessful = log.Commit(out var commit6Addr, out _, true, cookie6, 6); + commitSuccessful = log.CommitStrongly(out var commit6Addr, out _, true, cookie6, 6); Assert.IsTrue(commitSuccessful); var recoveredLog = new FasterLog(logSettings, 1); From 52d1bfde64978db57f49033aa194c2211b13e1f7 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Fri, 29 Oct 2021 18:39:01 +0000 Subject: [PATCH 18/33] fix bug with readonly log allocator initialization --- cs/src/core/Index/FasterLog/FasterLog.cs | 28 +++++++++--------------- 1 file changed, 10 insertions(+), 18 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 3da535dd3..9619b38ef 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1275,6 +1275,8 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co allocator.Initialize(); CommittedUntilAddress = Constants.kFirstValidAddress; beginAddress = allocator.BeginAddress; + if (readOnlyMode) + allocator.HeadAddress = long.MaxValue; return; } @@ -1293,6 +1295,8 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co cookie = info.Cookie; commitNum = info.CommitNum; beginAddress = allocator.BeginAddress; + if (readOnlyMode) + allocator.HeadAddress = long.MaxValue; } private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) @@ -1354,6 +1358,8 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary @@ -1393,6 +1399,8 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary, byte[]>(new Dictionary(), null); } @@ -1411,27 +1419,11 @@ private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary headAddress) - headAddress = info.BeginAddress; - - if (headAddress == 0) - headAddress = Constants.kFirstValidAddress; - } - - return true; - } - private Dictionary CompleteRestoreFromCommit(FasterLogRecoveryInfo info) { CommittedUntilAddress = info.UntilAddress; From ed3ea9e5de5c0fe97eaae5a78dbcc8a78cb45552 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Fri, 29 Oct 2021 20:47:39 +0000 Subject: [PATCH 19/33] disable fast commit temporarily for CI --- cs/src/core/Index/FasterLog/FasterLog.cs | 2 +- cs/test/FasterLogTests.cs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 9619b38ef..95699d43c 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -185,7 +185,6 @@ private FasterLog(FasterLogSettings logSettings, bool oldCommitManager) logSettings.GetLogSettings(), null, null, epoch, CommitCallback); allocator.Initialize(); - // TODO(Tianyu): Move to track commit record tail rather than user tail beginAddress = allocator.BeginAddress; // FasterLog is used as a read-only iterator @@ -1122,6 +1121,7 @@ private void SerialCommitCallbackWorker(CommitInfo commitInfo) if (latestCommit.FastForwardAllowed) WriteCommitMetadata(latestCommit); } + // TODO(Tianyu): Can invoke earlier in the case of fast commit var _commitTcs = commitTcs; commitTcs = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); var lci = new LinkedCommitInfo diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 4c1b2ff4f..4f9e0e719 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -804,13 +804,13 @@ internal class FasterLogCustomCommitTests : FasterLogTestBase [Test] [Category("FasterLog")] [Category("Smoke")] - public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) + public void FasterLogSimpleCommitCookieTest() { var cookie = new byte[100]; new Random().NextBytes(cookie); device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = fastCommit}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = false}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; From 710ade6627742723bf651a85fd23db95cf04272d Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 31 Oct 2021 15:39:50 +0000 Subject: [PATCH 20/33] fix MLSD and mem device fast commit behavior --- cs/src/core/Index/FasterLog/FasterLog.cs | 3 - .../core/Index/FasterLog/FasterLogIterator.cs | 114 +++++++++++------- cs/test/FasterLogFastCommitTests.cs | 87 +++++++++++++ cs/test/FasterLogTests.cs | 7 +- cs/test/TestUtils.cs | 10 +- 5 files changed, 165 insertions(+), 56 deletions(-) create mode 100644 cs/test/FasterLogFastCommitTests.cs diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 95699d43c..2c10925ac 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1711,9 +1711,6 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool CommitMetadataOnly(ref info, spinWait); } epoch.Suspend(); - // No need to trigger a manual commit even if no flush is triggered here -- if someone flushed the commit - // record between our last check of FlushedUntilAddress and now, their flush callback would observe - // this commitRecord in outstandingCommitRecords and commit it for us return true; } diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 36cf6502f..146c75ee9 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -203,19 +203,41 @@ public unsafe bool GetNext(out byte[] entry, out int entryLength, out long curre return false; } epoch.Resume(); - // Continue looping until we find a record that is not a commit record - while (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress, - out var isCommitRecord)) + while (true) { + long physicalAddress; + bool isCommitRecord; + try + { + var hasNext = GetNextInternal(out physicalAddress, out entryLength, out currentAddress, + out nextAddress, + out isCommitRecord); + if (!hasNext) + { + entry = default; + epoch.Suspend(); + return false; + } + } + catch (Exception) + { + // Throw upwards, but first, suspend the epoch we are in + epoch.Suspend(); + throw; + } + if (isCommitRecord) continue; - + if (getMemory != null) { // Use user delegate to allocate memory entry = getMemory(entryLength); if (entry.Length < entryLength) + { + epoch.Suspend(); throw new FasterException("Byte array provided has invalid length"); + } } else { @@ -224,15 +246,11 @@ public unsafe bool GetNext(out byte[] entry, out int entryLength, out long curre } fixed (byte* bp = entry) - Buffer.MemoryCopy((void*)(headerSize + physicalAddress), bp, entryLength, entryLength); + Buffer.MemoryCopy((void*) (headerSize + physicalAddress), bp, entryLength, entryLength); epoch.Suspend(); return true; } - - entry = default; - epoch.Suspend(); - return false; } /// @@ -269,12 +287,33 @@ public unsafe bool GetNext(MemoryPool pool, out IMemoryOwner entry, } epoch.Resume(); - // Continue looping until we find a record that is not a commit record - while (GetNextInternal(out long physicalAddress, out entryLength, out currentAddress, out nextAddress, out var isCommitRecord)) + while (true) { + long physicalAddress; + bool isCommitRecord; + try + { + var hasNext = GetNextInternal(out physicalAddress, out entryLength, out currentAddress, + out nextAddress, + out isCommitRecord); + if (!hasNext) + { + entry = default; + entryLength = default; + epoch.Suspend(); + return false; + } + } + catch (Exception) + { + // Throw upwards, but first, suspend the epoch we are in + epoch.Suspend(); + throw; + } + if (isCommitRecord) continue; - + entry = pool.Rent(entryLength); fixed (byte* bp = &entry.Memory.Span.GetPinnableReference()) @@ -283,12 +322,6 @@ public unsafe bool GetNext(MemoryPool pool, out IMemoryOwner entry, epoch.Suspend(); return true; } - - - entry = default; - entryLength = default; - epoch.Suspend(); - return false; } /// @@ -393,23 +426,12 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c foundCommit = true; byte[] entry; - if (getMemory != null) - { - // Use user delegate to allocate memory - entry = getMemory(entryLength); - if (entry.Length < entryLength) - throw new FasterException("Byte array provided has invalid length"); - } - else - { - // We allocate a byte array from heap - entry = new byte[entryLength]; - } - + // We allocate a byte array from heap + entry = new byte[entryLength]; fixed (byte* bp = entry) Buffer.MemoryCopy((void*) (headerSize + physicalAddress), bp, entryLength, entryLength); info.Initialize(new BinaryReader(new MemoryStream(entry))); - + // If we have already found the commit number we are looking for, can stop early if (info.CommitNum == commitNum) break; } @@ -418,7 +440,11 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c { // If we are here --- simply stop scanning because we ran into an incomplete entry } - epoch.Suspend(); + finally + { + epoch.Suspend(); + } + if (info.CommitNum == commitNum) return true; // User wants any commie @@ -489,19 +515,23 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt // Get and check entry length entryLength = fasterLog.GetLength((byte*)physicalAddress); + // We may encounter zeroed out bits at the end of page in a normal log, therefore, we need to check + // whether that is the case if (entryLength == 0) { - // We are likely at end of page, skip to next - currentAddress = (1 + (currentAddress >> allocator.LogPageSizeBits)) << allocator.LogPageSizeBits; - - Utility.MonotonicUpdate(ref nextAddress, currentAddress, out _); - - if (0 != fasterLog.GetChecksum((byte*)physicalAddress)) + // If zeroed out field is at page start, we encountered an uninitialized page and should signal up + var pageOffset = currentAddress & ((1 << allocator.LogPageSizeBits) - 1); + if (pageOffset == 0) { - epoch.Suspend(); var curPage = currentAddress >> allocator.LogPageSizeBits; - throw new FasterException("Invalid checksum found during scan, skipping page " + curPage); + throw new FasterException("Uninitialized page found during scan at page " + curPage); } + + // Otherwise, we must assume that zeroed out bits are due to page end and skip forward to the next + // page. If that's not the case, next iteration of the loop will either hit EOF exception or a + // blank page, and propagate failure upwards appropriately + currentAddress = (1 + (currentAddress >> allocator.LogPageSizeBits)) << allocator.LogPageSizeBits; + Utility.MonotonicUpdate(ref nextAddress, currentAddress, out _); continue; } @@ -518,7 +548,6 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt currentAddress = (1 + (currentAddress >> allocator.LogPageSizeBits)) << allocator.LogPageSizeBits; if (Utility.MonotonicUpdate(ref nextAddress, currentAddress, out _)) { - epoch.Suspend(); throw new FasterException("Invalid length of record found: " + entryLength + " at address " + currentAddress + ", skipping page"); } continue; @@ -533,7 +562,6 @@ private unsafe bool GetNextInternal(out long physicalAddress, out int entryLengt currentAddress = (1 + (currentAddress >> allocator.LogPageSizeBits)) << allocator.LogPageSizeBits; if (Utility.MonotonicUpdate(ref nextAddress, currentAddress, out _)) { - epoch.Suspend(); throw new FasterException("Invalid checksum found during scan, skipping page " + curPage); } continue; diff --git a/cs/test/FasterLogFastCommitTests.cs b/cs/test/FasterLogFastCommitTests.cs new file mode 100644 index 000000000..b787838c4 --- /dev/null +++ b/cs/test/FasterLogFastCommitTests.cs @@ -0,0 +1,87 @@ +using System; +using System.IO; +using FASTER.core; +using FASTER.test.recovery; +using NUnit.Framework; + +namespace FASTER.test +{ + [TestFixture] + internal class FasterLogFastCommitTests : FasterLogTestBase + { + [SetUp] + public void Setup() => base.BaseSetup(false, false); + + [TearDown] + public void TearDown() => base.BaseTearDown(); + + [Test] + [Category("FasterLog")] + [Category("Smoke")] + public void FasterLogSimpleFastCommitTest([Values] TestUtils.DeviceType deviceType) + { + var cookie = new byte[100]; + new Random().NextBytes(cookie); + + device = TestUtils.CreateTestDevice(deviceType, "fasterlog.log", deleteOnClose: true); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = true}; + log = new FasterLog(logSettings); + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var cookie1 = new byte[100]; + new Random().NextBytes(cookie1); + var commitSuccessful = log.CommitStrongly(out var commit1Addr, out _, true, cookie1, 1); + Assert.IsTrue(commitSuccessful); + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var cookie2 = new byte[100]; + new Random().NextBytes(cookie2); + commitSuccessful = log.CommitStrongly(out var commit2Addr, out _, true, cookie2, 2); + Assert.IsTrue(commitSuccessful); + + for (int i = 0; i < numEntries; i++) + { + log.Enqueue(entry); + } + + var cookie6 = new byte[100]; + new Random().NextBytes(cookie6); + commitSuccessful = log.CommitStrongly(out var commit6Addr, out _, true, cookie6, 6); + Assert.IsTrue(commitSuccessful); + + // be a deviant and remove commit metadata files + manager.PurgeAll(); + + // Recovery should still work + var recoveredLog = new FasterLog(logSettings, 1); + Assert.AreEqual(cookie1, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit1Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + + recoveredLog = new FasterLog(logSettings, 2); + Assert.AreEqual(cookie2, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit2Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + + // Default argument should recover to most recent + recoveredLog = new FasterLog(logSettings); + Assert.AreEqual(cookie6, recoveredLog.RecoveredCookie); + Assert.AreEqual(commit6Addr, recoveredLog.TailAddress); + recoveredLog.Dispose(); + } + + + } +} \ No newline at end of file diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 4f9e0e719..12ba5dab4 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -21,7 +21,6 @@ internal class FasterLogStandAloneTests [Test] [Category("FasterLog")] [Category("Smoke")] - public void TestDisposeReleasesFileLocksWithInprogressCommit([Values] TestUtils.DeviceType deviceType) { string path = TestUtils.MethodTestDir + "/"; @@ -804,13 +803,13 @@ internal class FasterLogCustomCommitTests : FasterLogTestBase [Test] [Category("FasterLog")] [Category("Smoke")] - public void FasterLogSimpleCommitCookieTest() + public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) { var cookie = new byte[100]; new Random().NextBytes(cookie); device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); - var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = false}; + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = fastCommit}; log = new FasterLog(logSettings); byte[] entry = new byte[entryLength]; @@ -828,7 +827,7 @@ public void FasterLogSimpleCommitCookieTest() Assert.AreEqual(cookie, recoveredLog.RecoveredCookie); recoveredLog.Dispose(); } - + [Test] [Category("FasterLog")] public void FasterLogManualCommitTest() diff --git a/cs/test/TestUtils.cs b/cs/test/TestUtils.cs index 1e4200c84..da6b80a53 100644 --- a/cs/test/TestUtils.cs +++ b/cs/test/TestUtils.cs @@ -90,7 +90,7 @@ public enum DeviceType LocalMemory } - internal static IDevice CreateTestDevice(DeviceType testDeviceType, string filename, int latencyMs = 20) // latencyMs works only for DeviceType = LocalMemory + internal static IDevice CreateTestDevice(DeviceType testDeviceType, string filename, int latencyMs = 20, bool deleteOnClose = false) // latencyMs works only for DeviceType = LocalMemory { IDevice device = null; bool preallocateFile = false; @@ -98,9 +98,7 @@ public enum DeviceType bool recoverDevice = false; bool useIoCompletionPort = false; bool disableFileBuffering = true; - - bool deleteOnClose = false; - + switch (testDeviceType) { #if WINDOWS @@ -112,7 +110,7 @@ public enum DeviceType break; case DeviceType.EmulatedAzure: IgnoreIfNotRunningAzureTests(); - device = new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, Path.GetFileName(filename), deleteOnClose: false); + device = new AzureStorageDevice(AzureEmulatedStorageString, AzureTestContainer, AzureTestDirectory, Path.GetFileName(filename), deleteOnClose: deleteOnClose); break; #endif case DeviceType.MLSD: @@ -120,7 +118,7 @@ public enum DeviceType break; // Emulated higher latency storage device - takes a disk latency arg (latencyMs) and emulates an IDevice using main memory, serving data at specified latency case DeviceType.LocalMemory: - device = new LocalMemoryDevice(1L << 26, 1L << 22, 2, sector_size: 512, latencyMs: latencyMs, fileName: filename); // 64 MB (1L << 26) is enough for our test cases + device = new LocalMemoryDevice(1L << 30, 1L << 30, 2, sector_size: 512, latencyMs: latencyMs, fileName: filename); // 64 MB (1L << 26) is enough for our test cases break; } From d841835bf01c6bfad2a32d6c0b1631dcc7ef073c Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 31 Oct 2021 17:58:33 +0000 Subject: [PATCH 21/33] fix AzureStorageDevice exception type and remove self-termination on failure --- cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs index 9954c3338..cb69d2053 100644 --- a/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs +++ b/cs/src/devices/AzureStorageDevice/AzureStorageDevice.cs @@ -182,7 +182,7 @@ public void PurgeAll() { foreach (var entry in blobs) { - entry.Value.PageBlob.Delete(); + entry.Value.PageBlob?.Delete(); } } @@ -361,8 +361,8 @@ public override unsafe void ReadAsync(int segmentId, ulong sourceAddress, IntPtr { var nonLoadedBlob = this.blobDirectory.GetPageBlobReference(GetSegmentBlobName(segmentId)); var exception = new InvalidOperationException("Attempt to read a non-loaded segment"); - this.BlobManager?.HandleBlobError(nameof(ReadAsync), exception.Message, nonLoadedBlob?.Name, exception, true); - throw exception; + this.BlobManager?.HandleBlobError(nameof(ReadAsync), exception.Message, nonLoadedBlob?.Name, exception, false); + throw new FasterException(exception.Message, exception); } var t = this.ReadFromBlobUnsafeAsync(blobEntry.PageBlob, (long)sourceAddress, (long)destinationAddress, readLength); From 76f4cec2c5621d381b8b082684712aa58efe2151 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Mon, 1 Nov 2021 13:51:36 +0000 Subject: [PATCH 22/33] test case on concurrent inserts --- cs/test/FasterLogFastCommitTests.cs | 59 ++++++++++++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/cs/test/FasterLogFastCommitTests.cs b/cs/test/FasterLogFastCommitTests.cs index b787838c4..98cdbbd9c 100644 --- a/cs/test/FasterLogFastCommitTests.cs +++ b/cs/test/FasterLogFastCommitTests.cs @@ -1,5 +1,8 @@ using System; +using System.Collections.Generic; +using System.Diagnostics; using System.IO; +using System.Threading; using FASTER.core; using FASTER.test.recovery; using NUnit.Framework; @@ -81,7 +84,61 @@ public void FasterLogSimpleFastCommitTest([Values] TestUtils.DeviceType deviceTy Assert.AreEqual(commit6Addr, recoveredLog.TailAddress); recoveredLog.Dispose(); } - + [Test] + [Category("FasterLog")] + [Category("Smoke")] + public void CommitRecordBoundedGrowthTest([Values] TestUtils.DeviceType deviceType) + { + var cookie = new byte[100]; + new Random().NextBytes(cookie); + + device = TestUtils.CreateTestDevice(deviceType, "fasterlog.log", deleteOnClose: true); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = true}; + log = new FasterLog(logSettings); + + + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (int i = 0; i < 5 * numEntries; i++) + log.Enqueue(entry); + + // for comparison, insert some entries without any commit records + var referenceTailLength = log.TailAddress; + + var enqueueDone = new ManualResetEventSlim(); + var commitThreads = new List(); + // Make sure to not spin up too many commit threads, otherwise we might clog epochs and halt progress + for (var i = 0; i < Math.Max(1, Environment.ProcessorCount - 1); i++) + { + commitThreads.Add(new Thread(() => + { + // Otherwise, absolutely clog the commit pipeline + while (!enqueueDone.IsSet) + log.Commit(); + })); + } + + foreach (var t in commitThreads) + t.Start(); + for (int i = 0; i < 5 * numEntries; i++) + { + log.Enqueue(entry); + } + enqueueDone.Set(); + + foreach (var t in commitThreads) + t.Join(); + + + // TODO: Hardcoded constant --- if this number changes in FasterLogRecoverInfo, it needs to be updated here too + var commitRecordSize = 44; + var logTailGrowth = log.TailAddress - referenceTailLength; + // Check that we are not growing the log more than one commit record per user entry + Assert.IsTrue(logTailGrowth - referenceTailLength <= commitRecordSize * 5 * numEntries); + } } } \ No newline at end of file From a354b7006d1708a432d6c7bf4f7353f7938f2ec4 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Wed, 3 Nov 2021 18:59:04 +0000 Subject: [PATCH 23/33] nits and test fixes --- cs/src/core/Allocator/WorkQueueLIFO.cs | 12 ++++++--- cs/src/core/Index/FasterLog/FasterLog.cs | 13 +++++++--- .../core/Index/FasterLog/FasterLogIterator.cs | 2 ++ .../Index/FasterLog/FasterLogRecoveryInfo.cs | 26 ++++++++++++------- cs/test/FasterLogFastCommitTests.cs | 8 +++--- cs/test/FasterLogTests.cs | 4 +-- 6 files changed, 42 insertions(+), 23 deletions(-) diff --git a/cs/src/core/Allocator/WorkQueueLIFO.cs b/cs/src/core/Allocator/WorkQueueLIFO.cs index 42eb76a62..2997f9a06 100644 --- a/cs/src/core/Allocator/WorkQueueLIFO.cs +++ b/cs/src/core/Allocator/WorkQueueLIFO.cs @@ -12,7 +12,7 @@ namespace FASTER.core /// Shared work queue that ensures one worker at any given time. Uses LIFO ordering of work. /// /// - class WorkQueueLIFO + class WorkQueueLIFO : IDisposable { const int kMaxQueueSize = 1 << 30; readonly ConcurrentStack _queue; @@ -26,6 +26,12 @@ public WorkQueueLIFO(Action work) _count = 0; } + public void Dispose() + { + while (_count != 0) + Thread.Sleep(10); + } + /// /// Enqueue work item, take ownership of draining the work queue /// if needed @@ -54,17 +60,17 @@ public void EnqueueAndTryWork(T work, bool asTask) private void ProcessQueue() { - // Process items in qork queue + // Process items in work queue while (true) { while (_queue.TryPop(out var workItem)) { - Interlocked.Decrement(ref _count); try { _work(workItem); } catch { } + Interlocked.Decrement(ref _count); } int count = _count; diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 2c10925ac..e5b8fed99 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -210,6 +210,7 @@ public void Dispose() internal void TrueDispose() { + commitQueue.Dispose(); commitTcs.TrySetException(new ObjectDisposedException("Log has been disposed")); allocator.Dispose(); epoch.Dispose(); @@ -1236,6 +1237,9 @@ private bool LoadCommitMetadata(long commitNum, out FasterLogRecoveryInfo info) info.Initialize(r); } + if (info.CommitNum == -1) + info.CommitNum = commitNum; + return true; } @@ -1279,7 +1283,7 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co allocator.HeadAddress = long.MaxValue; return; } - + if (!readOnlyMode) { var headAddress = info.UntilAddress - allocator.GetOffsetInPage(info.UntilAddress); @@ -1288,6 +1292,7 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co if (headAddress == 0) headAddress = Constants.kFirstValidAddress; + allocator.RestoreHybridLog(info.BeginAddress, headAddress, info.UntilAddress, info.UntilAddress); } @@ -1693,7 +1698,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool } // Otherwise, move to set read-only tail and flush - epoch.Resume(); + // epoch.Resume(); if (allocator.ShiftReadOnlyToTail(out _, out _)) { @@ -1701,7 +1706,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { while (CommittedUntilAddress < commitTail) { - epoch.ProtectAndDrain(); + // epoch.ProtectAndDrain(); Thread.Yield(); } } @@ -1710,7 +1715,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { CommitMetadataOnly(ref info, spinWait); } - epoch.Suspend(); + // epoch.Suspend(); return true; } diff --git a/cs/src/core/Index/FasterLog/FasterLogIterator.cs b/cs/src/core/Index/FasterLog/FasterLogIterator.cs index 146c75ee9..a5124c64e 100644 --- a/cs/src/core/Index/FasterLog/FasterLogIterator.cs +++ b/cs/src/core/Index/FasterLog/FasterLogIterator.cs @@ -431,6 +431,8 @@ internal unsafe bool ScanForwardForCommit(ref FasterLogRecoveryInfo info, long c fixed (byte* bp = entry) Buffer.MemoryCopy((void*) (headerSize + physicalAddress), bp, entryLength, entryLength); info.Initialize(new BinaryReader(new MemoryStream(entry))); + + Debug.Assert(info.CommitNum != -1); // If we have already found the commit number we are looking for, can stop early if (info.CommitNum == commitNum) break; diff --git a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs index c61933183..9b14cbd3f 100644 --- a/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs +++ b/cs/src/core/Index/FasterLog/FasterLogRecoveryInfo.cs @@ -66,13 +66,16 @@ public void Initialize(BinaryReader reader) checkSum = reader.ReadInt64(); BeginAddress = reader.ReadInt64(); UntilAddress = reader.ReadInt64(); - CommitNum = reader.ReadInt64(); + if (version == 1) + CommitNum = reader.ReadInt64(); + else + CommitNum = -1; } catch (Exception e) { throw new FasterException("Unable to recover from previous commit. Inner exception: " + e.ToString()); } - if (version != 0) + if (version != 0 && version != 1) throw new FasterException("Invalid version found during commit recovery"); if (checkSum != (BeginAddress ^ UntilAddress)) @@ -93,15 +96,18 @@ public void Initialize(BinaryReader reader) Iterators.Add(reader.ReadString(), reader.ReadInt64()); } } - - try + + if (version == 1) { - count = reader.ReadInt32(); - } - catch { } + try + { + count = reader.ReadInt32(); + } + catch { } - if (count > 0) - Cookie = reader.ReadBytes(count); + if (count > 0) + Cookie = reader.ReadBytes(count); + } } /// @@ -120,7 +126,7 @@ public readonly byte[] ToByteArray() using MemoryStream ms = new(); using (BinaryWriter writer = new(ms)) { - writer.Write(0); // version + writer.Write(1); // version writer.Write(BeginAddress ^ UntilAddress); // checksum writer.Write(BeginAddress); writer.Write(UntilAddress); diff --git a/cs/test/FasterLogFastCommitTests.cs b/cs/test/FasterLogFastCommitTests.cs index 98cdbbd9c..db75b2f57 100644 --- a/cs/test/FasterLogFastCommitTests.cs +++ b/cs/test/FasterLogFastCommitTests.cs @@ -26,7 +26,8 @@ public void FasterLogSimpleFastCommitTest([Values] TestUtils.DeviceType deviceTy var cookie = new byte[100]; new Random().NextBytes(cookie); - device = TestUtils.CreateTestDevice(deviceType, "fasterlog.log", deleteOnClose: true); + var filename = path + "fastCommit" + deviceType.ToString() + ".log"; + device = TestUtils.CreateTestDevice(deviceType, filename, deleteOnClose: true); var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = true}; log = new FasterLog(logSettings); @@ -93,11 +94,10 @@ public void CommitRecordBoundedGrowthTest([Values] TestUtils.DeviceType deviceTy var cookie = new byte[100]; new Random().NextBytes(cookie); - device = TestUtils.CreateTestDevice(deviceType, "fasterlog.log", deleteOnClose: true); + var filename = path + "boundedGrowth" + deviceType.ToString() + ".log"; + device = TestUtils.CreateTestDevice(deviceType, filename, deleteOnClose: true); var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = true}; log = new FasterLog(logSettings); - - byte[] entry = new byte[entryLength]; for (int i = 0; i < entryLength; i++) diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 12ba5dab4..7420260bd 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -808,7 +808,7 @@ public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) var cookie = new byte[100]; new Random().NextBytes(cookie); - device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); + device = Devices.CreateLogDevice(path + "SimpleCommitCookie" + fastCommit + ".log", deleteOnClose: true); var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.PerEntry, LogCommitManager = manager, FastCommitMode = fastCommit}; log = new FasterLog(logSettings); @@ -832,7 +832,7 @@ public void FasterLogSimpleCommitCookieTest([Values] bool fastCommit) [Category("FasterLog")] public void FasterLogManualCommitTest() { - device = Devices.CreateLogDevice(path + "fasterlog.log", deleteOnClose: true); + device = Devices.CreateLogDevice(path + "logManualCommitTest.log", deleteOnClose: true); var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; log = new FasterLog(logSettings); From 6d73d3400a4b6e477a7ebd0a13131eb6ea97fc1e Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 4 Nov 2021 00:04:41 +0000 Subject: [PATCH 24/33] Re-enable epoch protection on CommitInternal --- cs/src/core/Index/FasterLog/FasterLog.cs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index e5b8fed99..32e296476 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1637,8 +1637,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool if (allowFastForward && (cookie != null || proposedCommitNum != -1)) throw new FasterException( "Fast forwarding a commit is only allowed when no cookie and not commit num is specified"); - - + var info = new FasterLogRecoveryInfo(); info.FastForwardAllowed = allowFastForward; @@ -1663,7 +1662,6 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool else // Invalid commit num return false; - if (fastCommitMode) { @@ -1698,7 +1696,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool } // Otherwise, move to set read-only tail and flush - // epoch.Resume(); + epoch.Resume(); if (allocator.ShiftReadOnlyToTail(out _, out _)) { @@ -1706,7 +1704,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { while (CommittedUntilAddress < commitTail) { - // epoch.ProtectAndDrain(); + epoch.ProtectAndDrain(); Thread.Yield(); } } @@ -1715,7 +1713,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { CommitMetadataOnly(ref info, spinWait); } - // epoch.Suspend(); + epoch.Suspend(); return true; } From dcb9e0462dad1807ee90d15fda83153fba941300 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 4 Nov 2021 17:12:48 +0000 Subject: [PATCH 25/33] Try to identify issue on CI --- cs/src/core/Allocator/ErrorList.cs | 5 +++++ cs/src/core/Epochs/LightEpoch.cs | 1 - 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/cs/src/core/Allocator/ErrorList.cs b/cs/src/core/Allocator/ErrorList.cs index a776e9af3..5e4a7e852 100644 --- a/cs/src/core/Allocator/ErrorList.cs +++ b/cs/src/core/Allocator/ErrorList.cs @@ -22,8 +22,10 @@ public uint CheckAndWait(long oldFlushedUntilAddress, long currentFlushedUntilAd { bool done = false; uint errorCode = 0; + uint numIterations = 0; while (!done) { + numIterations++; done = true; lock (errorList) { @@ -40,6 +42,9 @@ public uint CheckAndWait(long oldFlushedUntilAddress, long currentFlushedUntilAd } } } + + if (numIterations >= 1000000) + throw new FasterException("Stuck inside CheckAndWait"); } return errorCode; } diff --git a/cs/src/core/Epochs/LightEpoch.cs b/cs/src/core/Epochs/LightEpoch.cs index 74d9f72e6..e7abf3b10 100644 --- a/cs/src/core/Epochs/LightEpoch.cs +++ b/cs/src/core/Epochs/LightEpoch.cs @@ -346,7 +346,6 @@ public void BumpCurrentEpoch(Action onDrain) if (++j == 500) { j = 0; - Debug.WriteLine("Delay finding a free entry in the drain list"); } } } From 26f09fff29302d24e4020f7e352169d9206dd08b Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 4 Nov 2021 19:10:52 +0000 Subject: [PATCH 26/33] fix Linux bug? --- cs/test/FasterLogFastCommitTests.cs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cs/test/FasterLogFastCommitTests.cs b/cs/test/FasterLogFastCommitTests.cs index db75b2f57..0c7ab8ae5 100644 --- a/cs/test/FasterLogFastCommitTests.cs +++ b/cs/test/FasterLogFastCommitTests.cs @@ -65,8 +65,12 @@ public void FasterLogSimpleFastCommitTest([Values] TestUtils.DeviceType deviceTy commitSuccessful = log.CommitStrongly(out var commit6Addr, out _, true, cookie6, 6); Assert.IsTrue(commitSuccessful); + // Wait for all metadata writes to be complete to avoid a concurrent access exception + log.Dispose(); + log = null; + // be a deviant and remove commit metadata files - manager.PurgeAll(); + manager.RemoveAllCommits(); // Recovery should still work var recoveredLog = new FasterLog(logSettings, 1); @@ -139,6 +143,8 @@ public void CommitRecordBoundedGrowthTest([Values] TestUtils.DeviceType deviceTy var logTailGrowth = log.TailAddress - referenceTailLength; // Check that we are not growing the log more than one commit record per user entry Assert.IsTrue(logTailGrowth - referenceTailLength <= commitRecordSize * 5 * numEntries); + + manager.RemoveAllCommits(); } } } \ No newline at end of file From 86056a5be466af14b82fe460b7d713220874e914 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 4 Nov 2021 21:41:14 +0000 Subject: [PATCH 27/33] Remove debugging --- cs/src/core/Allocator/ErrorList.cs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/cs/src/core/Allocator/ErrorList.cs b/cs/src/core/Allocator/ErrorList.cs index 5e4a7e852..a776e9af3 100644 --- a/cs/src/core/Allocator/ErrorList.cs +++ b/cs/src/core/Allocator/ErrorList.cs @@ -22,10 +22,8 @@ public uint CheckAndWait(long oldFlushedUntilAddress, long currentFlushedUntilAd { bool done = false; uint errorCode = 0; - uint numIterations = 0; while (!done) { - numIterations++; done = true; lock (errorList) { @@ -42,9 +40,6 @@ public uint CheckAndWait(long oldFlushedUntilAddress, long currentFlushedUntilAd } } } - - if (numIterations >= 1000000) - throw new FasterException("Stuck inside CheckAndWait"); } return errorCode; } From f0bf05e5d0dc209cf279543b2cbb28ccbac52f59 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 7 Nov 2021 15:29:15 +0000 Subject: [PATCH 28/33] Ensure that full flush list error callbacks do not break FlushedUntilAddress tracking --- cs/src/core/Allocator/AllocatorBase.cs | 3 ++ cs/src/core/Epochs/LightEpoch.cs | 2 +- cs/test/LogShiftTailStressTest.cs | 64 ++++++++++++++++++++++++++ 3 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 cs/test/LogShiftTailStressTest.cs diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index 37f745f50..f2b5e3e7a 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1761,6 +1761,9 @@ public void AsyncFlushPages(long fromAddress, long untilAddress) } else { + // Because we are invoking the callback away from the usual codepath, need to externally + // ensure that flush address are updated in order + while (FlushedUntilAddress < asyncResult.fromAddress) Thread.Yield(); // Could not add to pending flush list, treat as a failed write AsyncFlushPageCallback(1, 0, asyncResult); } diff --git a/cs/src/core/Epochs/LightEpoch.cs b/cs/src/core/Epochs/LightEpoch.cs index d856a0530..e2773a8a5 100644 --- a/cs/src/core/Epochs/LightEpoch.cs +++ b/cs/src/core/Epochs/LightEpoch.cs @@ -345,8 +345,8 @@ public void BumpCurrentEpoch(Action onDrain) i = 0; if (++j == 500) { + // Spin until there is a free entry in the drain list j = 0; - Debug.WriteLine("Delay finding a free entry in the drain list"); } } } diff --git a/cs/test/LogShiftTailStressTest.cs b/cs/test/LogShiftTailStressTest.cs new file mode 100644 index 000000000..d5009600a --- /dev/null +++ b/cs/test/LogShiftTailStressTest.cs @@ -0,0 +1,64 @@ +using System; +using System.Collections.Generic; +using System.Threading; +using FASTER.core; +using NUnit.Framework; + +namespace FASTER.test +{ + [TestFixture] + internal class LogShiftTailStressTest : FasterLogTestBase + { + [SetUp] + public void Setup() => base.BaseSetup(); + + [TearDown] + public void TearDown() => base.BaseTearDown(); + + [Test] + [Category("FasterLog")] + public void FasterLogShiftTailStressTest() + { + // Get an excruciatingly slow storage device to maximize chance of clogging the flush pipeline + device = new LocalMemoryDevice(1L << 32, 1 << 30, 2, sector_size: 512, latencyMs: 50, fileName: "stress.log"); + var logSettings = new FasterLogSettings { LogDevice = device, LogChecksum = LogChecksumType.None, LogCommitManager = manager}; + log = new FasterLog(logSettings); + + byte[] entry = new byte[entryLength]; + for (int i = 0; i < entryLength; i++) + entry[i] = (byte)i; + + for (int i = 0; i < 5 * numEntries; i++) + log.Enqueue(entry); + + // for comparison, insert some entries without any commit records + var referenceTailLength = log.TailAddress; + + var enqueueDone = new ManualResetEventSlim(); + var commitThreads = new List(); + // Make sure to spin up many commit threads to expose lots of interleavings + for (var i = 0; i < 2 * Math.Max(1, Environment.ProcessorCount - 1); i++) + { + commitThreads.Add(new Thread(() => + { + // Otherwise, absolutely clog the commit pipeline + while (!enqueueDone.IsSet) + log.Commit(); + })); + } + + foreach (var t in commitThreads) + t.Start(); + for (int i = 0; i < 5 * numEntries; i++) + { + log.Enqueue(entry); + } + enqueueDone.Set(); + + foreach (var t in commitThreads) + t.Join(); + + // We expect the test to finish and not get stuck somewhere + } + } +} \ No newline at end of file From 5e03355c3e340a60072bb8991aee572fa65ad690 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Sun, 7 Nov 2021 18:27:29 +0000 Subject: [PATCH 29/33] Fix unclean shutdown of stress test --- cs/src/core/Allocator/WorkQueueLIFO.cs | 25 +++++++++++++++++++++---- cs/test/FasterLogFastCommitTests.cs | 2 -- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/cs/src/core/Allocator/WorkQueueLIFO.cs b/cs/src/core/Allocator/WorkQueueLIFO.cs index 2997f9a06..5c7ee8355 100644 --- a/cs/src/core/Allocator/WorkQueueLIFO.cs +++ b/cs/src/core/Allocator/WorkQueueLIFO.cs @@ -3,6 +3,8 @@ using System; using System.Collections.Concurrent; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; using System.Threading; using System.Threading.Tasks; @@ -17,19 +19,25 @@ class WorkQueueLIFO : IDisposable const int kMaxQueueSize = 1 << 30; readonly ConcurrentStack _queue; readonly Action _work; - int _count; + private int _count; + private bool _disposed; public WorkQueueLIFO(Action work) { _queue = new ConcurrentStack(); _work = work; _count = 0; + _disposed = false; } public void Dispose() { + _disposed = true; + // All future enqueue requests will no longer perform work after _disposed is set to true. while (_count != 0) - Thread.Sleep(10); + Thread.Yield(); + // After this point, any previous work must have completed. Even if another enqueue request manipulates the + // count field, they are guaranteed to see disposed and not enqueue any actual work. } /// @@ -38,16 +46,24 @@ public void Dispose() /// /// Work to enqueue /// Process work as separate task - public void EnqueueAndTryWork(T work, bool asTask) + /// whether the enqueue is successful. Enqueuing into a disposed WorkQueue will fail and the task will not be performed> + public bool EnqueueAndTryWork(T work, bool asTask) { Interlocked.Increment(ref _count); + if (_disposed) + { + // Remove self from count in case Dispose() is actively waiting for completion + Interlocked.Decrement(ref _count); + return false; + } + _queue.Push(work); // Try to take over work queue processing if needed while (true) { int count = _count; - if (count >= kMaxQueueSize) return; + if (count >= kMaxQueueSize) return true; if (Interlocked.CompareExchange(ref _count, count + kMaxQueueSize, count) == count) break; } @@ -56,6 +72,7 @@ public void EnqueueAndTryWork(T work, bool asTask) _ = Task.Run(() => ProcessQueue()); else ProcessQueue(); + return true; } private void ProcessQueue() diff --git a/cs/test/FasterLogFastCommitTests.cs b/cs/test/FasterLogFastCommitTests.cs index 0c7ab8ae5..62a830d34 100644 --- a/cs/test/FasterLogFastCommitTests.cs +++ b/cs/test/FasterLogFastCommitTests.cs @@ -143,8 +143,6 @@ public void CommitRecordBoundedGrowthTest([Values] TestUtils.DeviceType deviceTy var logTailGrowth = log.TailAddress - referenceTailLength; // Check that we are not growing the log more than one commit record per user entry Assert.IsTrue(logTailGrowth - referenceTailLength <= commitRecordSize * 5 * numEntries); - - manager.RemoveAllCommits(); } } } \ No newline at end of file From b5e726d7896496b25d063a38bc78f284384d18a6 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Mon, 8 Nov 2021 13:58:26 +0000 Subject: [PATCH 30/33] update device log test to scan until tail instead --- cs/test/DeviceFasterLogTests.cs | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/cs/test/DeviceFasterLogTests.cs b/cs/test/DeviceFasterLogTests.cs index e1fc8f55b..0b36c4974 100644 --- a/cs/test/DeviceFasterLogTests.cs +++ b/cs/test/DeviceFasterLogTests.cs @@ -111,7 +111,10 @@ private async ValueTask FasterLogTest1(LogChecksumType logChecksum, IDevice devi } log.Commit(true); - using (var iter = log.Scan(0, long.MaxValue)) + // MoveNextAsync() would hang at TailAddress, waiting for more entries (that we don't add). + // Note: If this happens and the test has to be canceled, there may be a leftover blob from the log.Commit(), because + // the log device isn't Dispose()d; the symptom is currently a numeric string format error in DefaultCheckpointNamingScheme. + using (var iter = log.Scan(0, log.TailAddress)) { var counter = new FasterLogTestBase.Counter(log); @@ -122,12 +125,6 @@ private async ValueTask FasterLogTest1(LogChecksumType logChecksum, IDevice devi { Assert.IsTrue(result.SequenceEqual(entry)); counter.IncrementAndMaybeTruncateUntil(nextAddress); - - // MoveNextAsync() would hang at TailAddress, waiting for more entries (that we don't add). - // Note: If this happens and the test has to be canceled, there may be a leftover blob from the log.Commit(), because - // the log device isn't Dispose()d; the symptom is currently a numeric string format error in DefaultCheckpointNamingScheme. - if (nextAddress == log.TailAddress) - break; } break; case FasterLogTestBase.IteratorType.AsyncMemoryOwner: @@ -136,12 +133,6 @@ private async ValueTask FasterLogTest1(LogChecksumType logChecksum, IDevice devi Assert.IsTrue(result.Memory.Span.ToArray().Take(entry.Length).SequenceEqual(entry)); result.Dispose(); counter.IncrementAndMaybeTruncateUntil(nextAddress); - - // MoveNextAsync() would hang at TailAddress, waiting for more entries (that we don't add). - // Note: If this happens and the test has to be canceled, there may be a leftover blob from the log.Commit(), because - // the log device isn't Dispose()d; the symptom is currently a numeric string format error in DefaultCheckpointNamingScheme. - if (nextAddress == log.TailAddress) - break; } break; case FasterLogTestBase.IteratorType.Sync: From dc6729f0c88bc6bd6f37f229576517d96c4a72c5 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 11 Nov 2021 19:17:52 +0000 Subject: [PATCH 31/33] change error code used for full flush list to better distinguish from others --- cs/src/core/Allocator/AllocatorBase.cs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cs/src/core/Allocator/AllocatorBase.cs b/cs/src/core/Allocator/AllocatorBase.cs index 0aa0599ce..3a089d3f8 100644 --- a/cs/src/core/Allocator/AllocatorBase.cs +++ b/cs/src/core/Allocator/AllocatorBase.cs @@ -1766,7 +1766,8 @@ public void AsyncFlushPages(long fromAddress, long untilAddress) // ensure that flush address are updated in order while (FlushedUntilAddress < asyncResult.fromAddress) Thread.Yield(); // Could not add to pending flush list, treat as a failed write - AsyncFlushPageCallback(1, 0, asyncResult); + // Use a special errorCode to convey that this is not from a syscall + AsyncFlushPageCallback(16000, 0, asyncResult); } } else From 1fb0144a1faee52bb49d0fc581b836f763cf8074 Mon Sep 17 00:00:00 2001 From: Tianyu Li Date: Thu, 11 Nov 2021 19:47:05 +0000 Subject: [PATCH 32/33] nits --- cs/src/core/Index/FasterLog/FasterLog.cs | 2 -- cs/test/FasterLogTests.cs | 8 ++++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 32e296476..4d5d6e973 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1306,8 +1306,6 @@ private void RestoreLatest(out Dictionary iterators, out byte[] co private void RestoreSpecificCommit(long requestedCommitNum, out Dictionary iterators, out byte[] cookie) { - if (!logCommitManager.PreciseCommitNumRecoverySupport()) - throw new FasterException("Given FasterLog does not support recovery to a precise commit num"); iterators = null; cookie = null; FasterLogRecoveryInfo info = new(); diff --git a/cs/test/FasterLogTests.cs b/cs/test/FasterLogTests.cs index 7420260bd..7a8712f24 100644 --- a/cs/test/FasterLogTests.cs +++ b/cs/test/FasterLogTests.cs @@ -880,6 +880,14 @@ public void FasterLogManualCommitTest() Assert.AreEqual(commit2Addr, recoveredLog.TailAddress); recoveredLog.Dispose(); + // recovering to a non-existent commit should throw FasterException + try + { + recoveredLog = new FasterLog(logSettings, 4); + Assert.Fail(); + } + catch (FasterException) {} + // Default argument should recover to most recent recoveredLog = new FasterLog(logSettings); Assert.AreEqual(cookie6, recoveredLog.RecoveredCookie); From 5e74ddff29b128d281bf2837350287fa209a6b4f Mon Sep 17 00:00:00 2001 From: Badrish Chandramouli Date: Sat, 13 Nov 2021 10:17:44 -0800 Subject: [PATCH 33/33] Minor updates. --- cs/src/core/Index/FasterLog/FasterLog.cs | 67 ++++++++++-------------- 1 file changed, 29 insertions(+), 38 deletions(-) diff --git a/cs/src/core/Index/FasterLog/FasterLog.cs b/cs/src/core/Index/FasterLog/FasterLog.cs index 4d5d6e973..3b3dd5eed 100644 --- a/cs/src/core/Index/FasterLog/FasterLog.cs +++ b/cs/src/core/Index/FasterLog/FasterLog.cs @@ -1,8 +1,6 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT license. -#pragma warning disable 0162 - using System; using System.Buffers; using System.Collections.Concurrent; @@ -19,7 +17,7 @@ namespace FASTER.core /// /// FASTER log /// - public class FasterLog : IDisposable + public sealed class FasterLog : IDisposable { private readonly BlittableAllocator allocator; private readonly LightEpoch epoch; @@ -33,13 +31,11 @@ public class FasterLog : IDisposable internal readonly bool readOnlyMode; internal readonly bool fastCommitMode; - private TaskCompletionSource commitTcs - = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); - private TaskCompletionSource refreshUncommittedTcs - = new TaskCompletionSource(TaskCreationOptions.RunContinuationsAsynchronously); + private TaskCompletionSource commitTcs = new(TaskCreationOptions.RunContinuationsAsynchronously); + private TaskCompletionSource refreshUncommittedTcs = new(TaskCreationOptions.RunContinuationsAsynchronously); // Offsets for all currently unprocessed commit records - private Queue<(long, FasterLogRecoveryInfo)> ongoingCommitRequests; + private readonly Queue<(long, FasterLogRecoveryInfo)> ongoingCommitRequests; private long commitNum; /// @@ -108,8 +104,7 @@ private TaskCompletionSource refreshUncommittedTcs /// /// Table of persisted iterators /// - internal readonly ConcurrentDictionary PersistedIterators - = new ConcurrentDictionary(); + internal readonly ConcurrentDictionary PersistedIterators = new(); /// /// Committed view of commitMetadataVersion @@ -498,24 +493,23 @@ public void Commit(bool spinWait = false) } /// - /// Issue commit request for log (until tail) with the given commitNum + /// Issue a strong commit request for log (until tail) with the given commitNum /// - /// the tail committed by this call + /// The tail committed by this call /// - /// a unique, monotonically increasing identifier for the commit that can be used to recover to exactly - /// his commit + /// A unique, monotonically increasing identifier for the commit that can be used to recover to exactly this commit /// - /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. + /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately /// - /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from + /// A custom piece of metadata to be associated with this commit. If commit is successful, any recovery from /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed /// - /// /// - /// proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with + /// + /// Proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with /// a non -1 value, commit is guaranteed to have the supplied identifier if commit call is successful /// - /// whether commit is successful + /// Whether commit is successful public bool CommitStrongly(out long commitTail, out long actualCommitNum, bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1) { return CommitInternal(out commitTail, out actualCommitNum, spinWait, false, cookie, proposedCommitNum); @@ -566,32 +560,27 @@ public async ValueTask> CommitAsync(Task /// Issue commit request for log (until tail) with the given commitNum /// - /// the tail committed by this call - /// - /// a unique, monotonically increasing identifier for the commit that can be used to recover to exactly - /// his commit - /// - /// If true, spin-wait until commit completes. Otherwise, issue commit and return immediately. /// - /// a custom piece of metadata to be associated with this commit. If commit is successful, any recovery from + /// A custom piece of metadata to be associated with this commit. If commit is successful, any recovery from /// this commit will recover the cookie in RecoveredCookie field. Note that cookies are not stored by FasterLog /// itself, so the user is responsible for tracking cookie content and supplying it to every commit call if needed /// - /// /// - /// proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with + /// + /// Proposal for the identifier to use for this commit, or -1 if the system should pick one. If supplied with /// a non -1 value, commit is guaranteed to have the supplied identifier if commit call is successful /// - /// whether commit is successful - public async ValueTask<(bool, long, long)> CommitStronglyAsync(bool spinWait = false, byte[] cookie = null, long proposedCommitNum = -1, CancellationToken token = default) + /// Cancellation token + /// Whether commit is successful, commit tail, and actual commit number + public async ValueTask<(bool success, long commitTail, long actualCommitNum)> CommitStronglyAsync(byte[] cookie = null, long proposedCommitNum = -1, CancellationToken token = default) { token.ThrowIfCancellationRequested(); var task = CommitTask; if (!CommitInternal(out var commitTail, out var actualCommitNum, false, false, cookie, proposedCommitNum)) - return ValueTuple.Create(false, commitTail, actualCommitNum); + return (false, commitTail, actualCommitNum); while (CommittedUntilAddress < commitTail || persistedCommitNum < actualCommitNum) { @@ -599,7 +588,7 @@ public async ValueTask> CommitAsync(Task @@ -996,7 +985,7 @@ public async ValueTask ReadRecordLengthAsync(long address, CancellationToke } [MethodImpl(MethodImplOptions.AggressiveInlining)] - private int Align(int length) + private static int Align(int length) { return (length + 3) & ~3; } @@ -1635,9 +1624,11 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool if (allowFastForward && (cookie != null || proposedCommitNum != -1)) throw new FasterException( "Fast forwarding a commit is only allowed when no cookie and not commit num is specified"); - - var info = new FasterLogRecoveryInfo(); - info.FastForwardAllowed = allowFastForward; + + var info = new FasterLogRecoveryInfo + { + FastForwardAllowed = allowFastForward + }; // This critical section serializes commit record creation / commit content generation and ensures that the // long address are sorted in outstandingCommitRecords. Ok because we do not expect heavy contention on the @@ -1665,7 +1656,7 @@ private bool CommitInternal(out long commitTail, out long actualCommitNum, bool { // Ok to retry in critical section, any concurrently invoked commit would block, but cannot progress // anyways if no record can be enqueued - while (!TryEnqueueCommitRecord(ref info)) {} + while (!TryEnqueueCommitRecord(ref info)) Thread.Yield(); commitTail = info.UntilAddress; } else